You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2014/08/18 19:25:38 UTC
[01/51] [abbrv] [streaming] Package refactor & cleanup
Repository: incubator-flink
Updated Branches:
refs/heads/master 8d7e33c3c -> 923b508d1
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index c63dbda..8f64cb4 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -29,7 +29,7 @@ import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.log4j.Level;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
index aa86c68..012c89a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -27,8 +27,8 @@ import java.util.Iterator;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SinkFunction;
-import org.apache.flink.streaming.api.function.SourceFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.junit.Test;
import org.apache.flink.api.java.functions.GroupReduceFunction;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
index 95bd0a2..893ec37 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
@@ -28,7 +28,7 @@ import java.util.Set;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.junit.Test;
import org.apache.flink.api.java.functions.FlatMapFunction;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
index 4a6c1e0..2cea800 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
@@ -30,8 +30,8 @@ import java.util.Set;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SinkFunction;
-import org.apache.flink.streaming.api.function.SourceFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.junit.Test;
import org.apache.flink.api.java.functions.MapFunction;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
index 03fa3c5..89a2c7c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
@@ -27,11 +27,10 @@ import java.util.Map;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SinkFunction;
-import org.apache.flink.streaming.api.function.SourceFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.junit.BeforeClass;
import org.junit.Test;
-
import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
index 4d84c8f..0bb722c 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
@@ -21,8 +21,7 @@ package org.apache.flink.streaming.examples.basictopology;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
index c12f24a..9c0638d 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
@@ -23,8 +23,7 @@ import java.util.Random;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.functions.FlatMapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple4;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
index 773d2bd..8a76d0a 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
@@ -19,8 +19,7 @@
package org.apache.flink.streaming.examples.iterative.collaborativefilter;
-import org.apache.flink.streaming.api.function.SinkFunction;
-
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple4;
public class CollaborativeFilteringSink extends
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
index 694e92a..4776b20 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
@@ -22,8 +22,7 @@ package org.apache.flink.streaming.examples.iterative.collaborativefilter;
import java.io.BufferedReader;
import java.io.FileReader;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java
index 82842b8..047e7bf 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java
@@ -19,8 +19,7 @@
package org.apache.flink.streaming.examples.iterative.kmeans;
-import org.apache.flink.streaming.api.function.SinkFunction;
-
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple3;
public class KMeansSink extends SinkFunction<Tuple3<Integer, Integer, Long>> {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java
index fc27ea9..86720a2 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java
@@ -21,8 +21,7 @@ package org.apache.flink.streaming.examples.iterative.kmeans;
import java.util.Random;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java
index 9c1ae2e..9b0e23a 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java
@@ -19,8 +19,7 @@
package org.apache.flink.streaming.examples.iterative.pagerank;
-import org.apache.flink.streaming.api.function.SinkFunction;
-
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple3;
public class PageRankSink extends SinkFunction<Tuple3<Integer, Float, Long>> {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
index 94adedd..3269856 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
@@ -23,7 +23,7 @@ import java.io.BufferedReader;
import java.io.FileReader;
import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.api.function.SourceFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
public class PageRankSource extends SourceFunction<Tuple3<Integer, Integer, Long>> {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java
index 7a1c345..0b10be0 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java
@@ -19,8 +19,7 @@
package org.apache.flink.streaming.examples.iterative.sssp;
-import org.apache.flink.streaming.api.function.SinkFunction;
-
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple3;
public class SSSPSink extends SinkFunction<Tuple3<Integer, Integer, Long>> {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
index 8bdd3c6..15be203 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
@@ -22,8 +22,7 @@ package org.apache.flink.streaming.examples.iterative.sssp;
import java.io.BufferedReader;
import java.io.FileReader;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java
index 6297781..1d87a20 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java
@@ -19,8 +19,7 @@
package org.apache.flink.streaming.examples.join;
-import org.apache.flink.streaming.api.function.SinkFunction;
-
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple3;
public class JoinSink extends SinkFunction<Tuple3<String, Integer, Integer>> {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java
index a5d50ad..d178033 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java
@@ -21,8 +21,7 @@ package org.apache.flink.streaming.examples.join;
import java.util.Random;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java
index 9d20358..3730534 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java
@@ -21,8 +21,7 @@ package org.apache.flink.streaming.examples.join;
import java.util.Random;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index 9c9ec7b..d855639 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -21,8 +21,7 @@ package org.apache.flink.streaming.examples.ml;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
index 0662c25..b4c97c3 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
@@ -25,8 +25,7 @@ import org.apache.commons.lang.ArrayUtils;
import org.apache.commons.math.stat.regression.OLSMultipleLinearRegression;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple1;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java
index 1f1f966..9e16ef8 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java
@@ -21,8 +21,7 @@ package org.apache.flink.streaming.examples.window.join;
import java.util.Random;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java
index bb1c126..dfac58d 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java
@@ -21,8 +21,7 @@ package org.apache.flink.streaming.examples.window.join;
import java.util.Random;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSink.java
index f2cec34..9c4547a 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSink.java
@@ -19,8 +19,7 @@
package org.apache.flink.streaming.examples.window.sum;
-import org.apache.flink.streaming.api.function.SinkFunction;
-
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple2;
public class WindowSumSink extends SinkFunction<Tuple2<Integer, Long>> {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSource.java
index 93deb93..12eda38 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSource.java
@@ -19,8 +19,7 @@
package org.apache.flink.streaming.examples.window.sum;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSink.java
index fd8e7b0..f8c009f 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSink.java
@@ -19,8 +19,7 @@
package org.apache.flink.streaming.examples.window.wordcount;
-import org.apache.flink.streaming.api.function.SinkFunction;
-
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple3;
public class WindowWordCountSink extends SinkFunction<Tuple3<String, Integer, Long>> {
[05/51] [abbrv] git commit: [streaming] Fixed StreamCollectorTest
Posted by se...@apache.org.
[streaming] Fixed StreamCollectorTest
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/11f62c1e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/11f62c1e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/11f62c1e
Branch: refs/heads/master
Commit: 11f62c1e32654ccda6d2ac1588cc4b6afa74e32f
Parents: c21f0e3
Author: ghermann <re...@gmail.com>
Authored: Mon Jul 21 11:23:32 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:14:13 2014 +0200
----------------------------------------------------------------------
.../streaming/api/collector/StreamCollectorTest.java | 10 ++++++----
.../streaming/api/streamcomponent/MockRecordWriter.java | 12 ++++++------
.../flink/streaming/util/MockRecordWriterFactory.java | 4 ++--
3 files changed, 14 insertions(+), 12 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/11f62c1e/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
index 8fc8704..7c7f593 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
@@ -19,27 +19,29 @@
package org.apache.flink.streaming.api.collector;
+import static org.junit.Assert.assertArrayEquals;
+
import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamcomponent.MockRecordWriter;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.MockRecordWriterFactory;
import org.junit.Test;
public class StreamCollectorTest {
-
-
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Test
public void testCollect() {
MockRecordWriter recWriter = MockRecordWriterFactory.create();
- StreamCollector collector = new StreamCollector(2, null);
+ StreamCollector<Tuple1<Integer>> collector = new StreamCollector<Tuple1<Integer>>(2, new SerializationDelegate<StreamRecord<Tuple1<Integer>>>(null));
collector.addOutput(recWriter, null);
collector.collect(new Tuple1<Integer>(3));
collector.collect(new Tuple1<Integer>(4));
collector.collect(new Tuple1<Integer>(5));
collector.collect(new Tuple1<Integer>(6));
+ assertArrayEquals(new Integer[] {3, 4, 5, 6}, recWriter.emittedRecords.toArray());
}
@Test
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/11f62c1e/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
index e4574f1..e99518c 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
@@ -22,26 +22,26 @@ package org.apache.flink.streaming.api.streamcomponent;
import java.util.ArrayList;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.runtime.operators.DataSourceTask;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.runtime.io.network.api.RecordWriter;
-public class MockRecordWriter extends RecordWriter<StreamRecord<Tuple1<Integer>>> {
+public class MockRecordWriter extends RecordWriter<SerializationDelegate<StreamRecord<Tuple1<Integer>>>> {
- public ArrayList<StreamRecord<Tuple1<Integer>>> emittedRecords;
+ public ArrayList<Integer> emittedRecords;
public MockRecordWriter(DataSourceTask<?> inputBase, Class<StreamRecord<Tuple1<Integer>>> outputClass) {
super(inputBase);
}
public boolean initList() {
- emittedRecords = new ArrayList<StreamRecord<Tuple1<Integer>>>();
+ emittedRecords = new ArrayList<Integer>();
return true;
}
@Override
- public void emit(StreamRecord<Tuple1<Integer>> record) {
- emittedRecords.add(record);
+ public void emit(SerializationDelegate<StreamRecord<Tuple1<Integer>>> record) {
+ emittedRecords.add(record.getInstance().getTuple().f0);
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/11f62c1e/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java
index ed018eb..4b85119 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java
@@ -22,8 +22,8 @@ package org.apache.flink.streaming.util;
import static org.mockito.Mockito.doCallRealMethod;
import static org.mockito.Mockito.mock;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamcomponent.MockRecordWriter;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.mockito.Mockito;
public class MockRecordWriterFactory {
@@ -33,7 +33,7 @@ public class MockRecordWriterFactory {
MockRecordWriter recWriter = mock(MockRecordWriter.class);
Mockito.when(recWriter.initList()).thenCallRealMethod();
- doCallRealMethod().when(recWriter).emit(Mockito.any(StreamRecord.class));
+ doCallRealMethod().when(recWriter).emit(Mockito.any(SerializationDelegate.class));
recWriter.initList();
[51/51] [abbrv] git commit: [streaming] Fixes for tests to support
parallel builds
Posted by se...@apache.org.
[streaming] Fixes for tests to support parallel builds
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/923b508d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/923b508d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/923b508d
Branch: refs/heads/master
Commit: 923b508d18fdf471c0c74a263bc6c93b9c4ecf59
Parents: 0163cfa
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Aug 18 18:48:55 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 18:48:55 2014 +0200
----------------------------------------------------------------------
.../streaming/api/datastream/DataStream.java | 1 -
.../api/environment/LocalStreamEnvironment.java | 3 -
.../environment/StreamExecutionEnvironment.java | 1 -
.../flink/streaming/util/ClusterUtil.java | 42 ++++++-------
.../apache/flink/streaming/api/IterateTest.java | 7 +--
.../apache/flink/streaming/api/PrintTest.java | 9 +--
.../flink/streaming/api/WriteAsCsvTest.java | 57 ++++++++++--------
.../flink/streaming/api/WriteAsTextTest.java | 62 +++++++++++---------
.../api/invokable/operator/BatchReduceTest.java | 2 -
.../api/invokable/operator/CoMapTest.java | 10 ++--
.../api/invokable/operator/FilterTest.java | 2 -
.../api/invokable/operator/FlatMapTest.java | 2 -
.../api/invokable/operator/MapTest.java | 2 -
.../streamcomponent/StreamComponentTest.java | 24 ++++----
14 files changed, 109 insertions(+), 115 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 243cc5e..ab14bc6 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -159,7 +159,6 @@ public abstract class DataStream<OUT> {
* The DataStreams to merge output with.
* @return The {@link MergedDataStream}.
*/
- @SuppressWarnings("unchecked")
public MergedDataStream<OUT> merge(DataStream<OUT>... streams) {
MergedDataStream<OUT> returnStream = new MergedDataStream<OUT>(this);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
index 16e84bb..828a566 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api.environment;
@@ -36,5 +34,4 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment {
ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(), getExecutionParallelism(),
memorySize);
}
-
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 9b74b40..68e2421 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -194,7 +194,6 @@ public abstract class StreamExecutionEnvironment {
* type of the returned stream
* @return The DataStream representing the elements.
*/
- @SuppressWarnings("unchecked")
public <OUT extends Serializable> DataStreamSource<OUT> fromElements(OUT... data) {
DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements");
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
index a6e842f..8815e78 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.util;
@@ -25,7 +23,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.client.minicluster.NepheleMiniCluster;
import org.apache.flink.client.program.Client;
-import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -49,7 +46,6 @@ public class ClusterUtil {
NepheleMiniCluster exec = new NepheleMiniCluster();
exec.setMemorySize(memorySize);
exec.setNumTaskTracker(numberOfTaskTrackers);
- Client client = new Client(new InetSocketAddress("localhost", 6498), configuration, ClusterUtil.class.getClassLoader());
if (LOG.isInfoEnabled()) {
LOG.info("Running on mini cluster");
@@ -58,11 +54,15 @@ public class ClusterUtil {
try {
exec.start();
+ Client client = new Client(new InetSocketAddress("localhost", exec.getJobManagerRpcPort()), configuration, ClusterUtil.class.getClassLoader());
client.run(jobGraph, true);
- exec.stop();
} catch (Exception e) {
throw new RuntimeException(e);
+ } finally {
+ try {
+ exec.stop();
+ } catch (Throwable t) {}
}
}
@@ -70,20 +70,22 @@ public class ClusterUtil {
runOnMiniCluster(jobGraph, numberOfTaskTrackers, -1);
}
- public static void runOnLocalCluster(JobGraph jobGraph, String IP, int port) {
- if (LOG.isInfoEnabled()) {
- LOG.info("Running on mini cluster");
- }
-
- Configuration configuration = jobGraph.getJobConfiguration();
-
- Client client = new Client(new InetSocketAddress(IP, port), configuration, ClusterUtil.class.getClassLoader());
-
- try {
- client.run(jobGraph, true);
- } catch (ProgramInvocationException e) {
- throw new RuntimeException(e);
- }
- }
+ // this one will not work easily any more, because we recently introduces concurrent test, which
+ // implies that the master RPC port becomes dynamic
+// public static void runOnLocalCluster(JobGraph jobGraph, String IP, int port) {
+// if (LOG.isInfoEnabled()) {
+// LOG.info("Running on mini cluster");
+// }
+//
+// Configuration configuration = jobGraph.getJobConfiguration();
+//
+// Client client = new Client(new InetSocketAddress(IP, port), configuration, ClusterUtil.class.getClassLoader());
+//
+// try {
+// client.run(jobGraph, true);
+// } catch (ProgramInvocationException e) {
+// throw new RuntimeException(e);
+// }
+// }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 9498b8e..da9de05 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api;
@@ -27,9 +25,8 @@ import org.apache.flink.streaming.api.datastream.IterativeDataStream;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.util.LogUtils;
import org.apache.flink.util.Collector;
-import org.apache.log4j.Level;
+import org.apache.flink.util.LogUtils;
import org.junit.Test;
public class IterateTest {
@@ -77,7 +74,7 @@ public class IterateTest {
@Test
public void test() throws Exception {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+ LogUtils.initializeDefaultTestConsoleLogger();
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
index 90fac36..cdcd993 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,15 +13,13 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.LogUtils;
-import org.apache.log4j.Level;
+import org.apache.flink.util.LogUtils;
import org.junit.Test;
public class PrintTest{
@@ -31,12 +28,10 @@ public class PrintTest{
@Test
public void test() throws Exception {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+ LogUtils.initializeDefaultTestConsoleLogger();
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
env.generateSequence(1, 10).print();
env.executeTest(MEMORYSIZE);
-
}
-
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
index 8e9475a..77dc9c8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
@@ -34,9 +34,14 @@ import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
+import org.apache.flink.util.LogUtils;
+import org.junit.BeforeClass;
import org.junit.Test;
public class WriteAsCsvTest {
+
+ private static final String PREFIX = System.getProperty("java.io.tmpdir") + "/" + WriteAsCsvTest.class.getSimpleName() + "_";
+
private static final long MEMORYSIZE = 32;
private static List<String> result1 = new ArrayList<String>();
@@ -111,61 +116,67 @@ public class WriteAsCsvTest {
}
}
+ @BeforeClass
+ public static void createFileCleanup() {
+ Runnable r = new Runnable() {
+
+ @Override
+ public void run() {
+ try { new File(PREFIX + "test1.txt").delete(); } catch (Throwable t) {}
+ try { new File(PREFIX + "test2.txt").delete(); } catch (Throwable t) {}
+ try { new File(PREFIX + "test3.txt").delete(); } catch (Throwable t) {}
+ try { new File(PREFIX + "test4.txt").delete(); } catch (Throwable t) {}
+ try { new File(PREFIX + "test5.txt").delete(); } catch (Throwable t) {}
+ }
+ };
+
+ Runtime.getRuntime().addShutdownHook(new Thread(r));
+ }
+
@Test
public void test() throws Exception {
+ LogUtils.initializeDefaultTestConsoleLogger();
+
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream1 = env.addSource(new MySource1(), 1).writeAsCsv(
- "test1.txt");
+ DataStream<Tuple1<Integer>> dataStream1 = env.addSource(new MySource1(), 1).writeAsCsv(PREFIX + "test1.txt");
fillExpected1();
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream2 = env.addSource(new MySource1(), 1).writeAsCsv(
- "test2.txt", 5);
+ DataStream<Tuple1<Integer>> dataStream2 = env.addSource(new MySource1(), 1).writeAsCsv(PREFIX + "test2.txt", 5);
fillExpected2();
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream3 = env.addSource(new MySource1(), 1).writeAsCsv(
- "test3.txt", 10);
+ DataStream<Tuple1<Integer>> dataStream3 = env.addSource(new MySource1(), 1).writeAsCsv(PREFIX + "test3.txt", 10);
fillExpected3();
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream4 = env.addSource(new MySource1(), 1).writeAsCsv(
- "test4.txt", 10, new Tuple1<Integer>(26));
+ DataStream<Tuple1<Integer>> dataStream4 = env.addSource(new MySource1(), 1).writeAsCsv(PREFIX + "test4.txt", 10, new Tuple1<Integer>(26));
fillExpected4();
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream5 = env.addSource(new MySource1(), 1).writeAsCsv(
- "test5.txt", 10, new Tuple1<Integer>(14));
+ DataStream<Tuple1<Integer>> dataStream5 = env.addSource(new MySource1(), 1).writeAsCsv(PREFIX + "test5.txt", 10, new Tuple1<Integer>(14));
fillExpected5();
env.executeTest(MEMORYSIZE);
- readFile("test1.txt", result1);
- readFile("test2.txt", result2);
- readFile("test3.txt", result3);
- readFile("test4.txt", result4);
- readFile("test5.txt", result5);
+ readFile(PREFIX + "test1.txt", result1);
+ readFile(PREFIX + "test2.txt", result2);
+ readFile(PREFIX + "test3.txt", result3);
+ readFile(PREFIX + "test4.txt", result4);
+ readFile(PREFIX + "test5.txt", result5);
assertTrue(expected1.equals(result1));
assertTrue(expected2.equals(result2));
assertTrue(expected3.equals(result3));
assertTrue(expected4.equals(result4));
assertTrue(expected5.equals(result5));
-
- new File("test1.txt").delete();
- new File("test2.txt").delete();
- new File("test3.txt").delete();
- new File("test4.txt").delete();
- new File("test5.txt").delete();
-
}
-
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
index 122979d..eea97ec 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api;
@@ -33,12 +31,15 @@ import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.streaming.util.LogUtils;
+import org.apache.flink.util.LogUtils;
import org.apache.flink.util.Collector;
-import org.apache.log4j.Level;
+import org.junit.BeforeClass;
import org.junit.Test;
public class WriteAsTextTest {
+
+ private static final String PREFIX = System.getProperty("java.io.tmpdir") + "/" + WriteAsTextTest.class.getSimpleName() + "_";
+
private static final long MEMORYSIZE = 32;
private static List<String> result1 = new ArrayList<String>();
@@ -112,64 +113,69 @@ public class WriteAsTextTest {
expected5.add("(" + i + ")");
}
}
+
+ @BeforeClass
+ public static void createFileCleanup() {
+ Runnable r = new Runnable() {
+
+ @Override
+ public void run() {
+ try { new File(PREFIX + "test1.txt").delete(); } catch (Throwable t) {}
+ try { new File(PREFIX + "test2.txt").delete(); } catch (Throwable t) {}
+ try { new File(PREFIX + "test3.txt").delete(); } catch (Throwable t) {}
+ try { new File(PREFIX + "test4.txt").delete(); } catch (Throwable t) {}
+ try { new File(PREFIX + "test5.txt").delete(); } catch (Throwable t) {}
+ }
+ };
+
+ Runtime.getRuntime().addShutdownHook(new Thread(r));
+ }
@Test
public void test() throws Exception {
+ LogUtils.initializeDefaultTestConsoleLogger();
+
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream1 = env.addSource(new MySource1(), 1).writeAsText(
- "test1.txt");
+ DataStream<Tuple1<Integer>> dataStream1 = env.addSource(new MySource1(), 1).writeAsText(PREFIX + "test1.txt");
fillExpected1();
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream2 = env.addSource(new MySource1(), 1).writeAsText(
- "test2.txt", 5);
+ DataStream<Tuple1<Integer>> dataStream2 = env.addSource(new MySource1(), 1).writeAsText(PREFIX + "test2.txt", 5);
fillExpected2();
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream3 = env.addSource(new MySource1(), 1).writeAsText(
- "test3.txt", 10);
+ DataStream<Tuple1<Integer>> dataStream3 = env.addSource(new MySource1(), 1).writeAsText(PREFIX + "test3.txt", 10);
fillExpected3();
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream4 = env.addSource(new MySource1(), 1).writeAsText(
- "test4.txt", 10, new Tuple1<Integer>(26));
+ DataStream<Tuple1<Integer>> dataStream4 = env.addSource(new MySource1(), 1).writeAsText(PREFIX + "test4.txt", 10, new Tuple1<Integer>(26));
fillExpected4();
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream5 = env.addSource(new MySource1(), 1).writeAsText(
- "test5.txt", 10, new Tuple1<Integer>(14));
+ DataStream<Tuple1<Integer>> dataStream5 = env.addSource(new MySource1(), 1).writeAsText(PREFIX + "test5.txt", 10, new Tuple1<Integer>(14));
fillExpected5();
env.executeTest(MEMORYSIZE);
- readFile("test1.txt", result1);
- readFile("test2.txt", result2);
- readFile("test3.txt", result3);
- readFile("test4.txt", result4);
- readFile("test5.txt", result5);
+ readFile(PREFIX + "test1.txt", result1);
+ readFile(PREFIX + "test2.txt", result2);
+ readFile(PREFIX + "test3.txt", result3);
+ readFile(PREFIX + "test4.txt", result4);
+ readFile(PREFIX + "test5.txt", result5);
assertEquals(expected1,result1);
assertEquals(expected2,result2);
assertEquals(expected3,result3);
assertEquals(expected4,result4);
assertEquals(expected5,result5);
-
- new File("test1.txt").delete();
- new File("test2.txt").delete();
- new File("test3.txt").delete();
- new File("test4.txt").delete();
- new File("test5.txt").delete();
-
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
index 63ba627..85534ad 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api.invokable.operator;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
index 068efe2..e24362e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api.invokable.operator;
@@ -28,8 +26,8 @@ import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.util.LogUtils;
-import org.apache.log4j.Level;
+import org.apache.flink.util.LogUtils;
+
import org.junit.Assert;
import org.junit.Test;
@@ -65,7 +63,9 @@ public class CoMapTest implements Serializable {
@Test
public void multipleInputTest() {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+
+ LogUtils.initializeDefaultTestConsoleLogger();
+
expected.add("a");
expected.add("b");
expected.add("c");
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
index 152f992..0f4a5a3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api.invokable.operator;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
index fe367d3..4d84435 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api.invokable.operator;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
index e3c7cb7..8096a88 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api.invokable.operator;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/923b508d/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
index 545169d..b16ffee 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
@@ -1,5 +1,4 @@
/**
- *
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
@@ -14,7 +13,6 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
- *
*/
package org.apache.flink.streaming.api.streamcomponent;
@@ -27,23 +25,22 @@ import java.util.Map;
import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
-import org.junit.BeforeClass;
+import org.apache.flink.util.LogUtils;
import org.junit.Test;
public class StreamComponentTest {
@SuppressWarnings("unused")
- private static final int PARALLELISM = 1;
+ private static final int PARALLELISM = 1;
private static final int SOURCE_PARALELISM = 1;
private static final long MEMORYSIZE = 32;
- public static Map<Integer, Integer> data = new HashMap<Integer, Integer>();
+ private static Map<Integer, Integer> data = new HashMap<Integer, Integer>();
public static class MySource implements SourceFunction<Tuple1<Integer>> {
private static final long serialVersionUID = 1L;
@@ -93,20 +90,19 @@ public class StreamComponentTest {
}
}
- @SuppressWarnings("unused")
- @BeforeClass
- public static void runStream() {
+ @Test
+ public void runStream() {
+
+ LogUtils.initializeDefaultTestConsoleLogger();
+
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(SOURCE_PARALELISM);
- DataStream<Tuple2<Integer, Integer>> oneTask = env
+ env
.addSource(new MySource(), SOURCE_PARALELISM).map(new MyTask())
.addSink(new MySink());
env.executeTest(MEMORYSIZE);
- }
-
- @Test
- public void test() {
+
assertEquals(10, data.keySet().size());
for (Integer k : data.keySet()) {
[40/51] [abbrv] git commit: [streaming] API update with more
differentiated DataStream types and javadoc + several fixes
Posted by se...@apache.org.
[streaming] API update with more differentiated DataStream types and javadoc + several fixes
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/d56d48f1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/d56d48f1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/d56d48f1
Branch: refs/heads/master
Commit: d56d48f1a1933f138f5d97f42b71aa64fad6bc6f
Parents: e73ea29
Author: gyfora <gy...@gmail.com>
Authored: Tue Aug 5 15:00:55 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:16 2014 +0200
----------------------------------------------------------------------
.../streaming/connectors/flume/FlumeSink.java | 2 +-
.../streaming/connectors/flume/FlumeSource.java | 2 +-
.../connectors/flume/FlumeTopology.java | 4 +-
.../streaming/connectors/kafka/KafkaSource.java | 2 +-
.../connectors/kafka/KafkaTopology.java | 4 +-
.../connectors/rabbitmq/RMQSource.java | 2 +-
.../connectors/rabbitmq/RMQTopology.java | 4 +-
.../connectors/twitter/TwitterLocal.java | 6 +-
.../connectors/twitter/TwitterStreaming.java | 4 +-
.../apache/flink/streaming/api/DataStream.java | 975 -------------------
.../streaming/api/IterativeDataStream.java | 97 --
.../streaming/api/LocalStreamEnvironment.java | 40 -
.../flink/streaming/api/NamedDataStream.java | 33 -
.../streaming/api/RemoteStreamEnvironment.java | 107 --
.../api/SingleInputStreamOperator.java | 46 -
.../flink/streaming/api/SplitDataStream.java | 47 -
.../flink/streaming/api/StreamConfig.java | 17 +-
.../api/StreamExecutionEnvironment.java | 410 --------
.../flink/streaming/api/StreamOperator.java | 44 -
.../streaming/api/TwoInputStreamOperator.java | 49 -
.../api/collector/DirectedStreamCollector.java | 11 +-
.../streaming/api/collector/OutputSelector.java | 16 +-
.../api/collector/StreamCollector.java | 26 +-
.../streaming/api/datastream/CoDataStream.java | 132 +++
.../api/datastream/ConnectedDataStream.java | 98 ++
.../streaming/api/datastream/DataStream.java | 852 ++++++++++++++++
.../api/datastream/DataStreamSink.java | 45 +
.../api/datastream/DataStreamSource.java | 39 +
.../api/datastream/IterativeDataStream.java | 103 ++
.../datastream/SingleOutputStreamOperator.java | 148 +++
.../api/datastream/SplitDataStream.java | 70 ++
.../api/environment/LocalStreamEnvironment.java | 40 +
.../environment/RemoteStreamEnvironment.java | 107 ++
.../environment/StreamExecutionEnvironment.java | 413 ++++++++
.../api/function/co/CoMapFunction.java | 11 +
.../api/function/co/RichCoMapFunction.java | 13 +
.../AbstractStreamComponent.java | 2 +-
.../apache/flink/streaming/api/IterateTest.java | 4 +
.../apache/flink/streaming/api/PrintTest.java | 2 +
.../flink/streaming/api/WriteAsCsvTest.java | 3 +
.../flink/streaming/api/WriteAsTextTest.java | 3 +
.../api/collector/DirectedOutputTest.java | 29 +-
.../api/collector/StreamCollectorTest.java | 9 +-
.../api/invokable/operator/BatchReduceTest.java | 6 +-
.../api/invokable/operator/CoMapTest.java | 15 +-
.../api/invokable/operator/FilterTest.java | 2 +-
.../api/invokable/operator/FlatMapTest.java | 6 +-
.../api/invokable/operator/MapTest.java | 6 +-
.../streamcomponent/StreamComponentTest.java | 6 +-
.../examples/basictopology/BasicTopology.java | 4 +-
.../examples/cellinfo/CellInfoLocal.java | 4 +-
.../examples/iterative/kmeans/KMeansLocal.java | 2 +-
.../iterative/pagerank/PageRankLocal.java | 2 +-
.../examples/iterative/sssp/SSSPLocal.java | 2 +-
.../streaming/examples/join/JoinLocal.java | 5 +-
.../ml/IncrementalLearningSkeleton.java | 4 +-
.../streaming/examples/ml/IncrementalOLS.java | 4 +-
.../examples/window/join/WindowJoinLocal.java | 4 +-
.../examples/wordcount/WordCountLocal.java | 5 +-
59 files changed, 2189 insertions(+), 1959 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
index 69e34e6..349a9a5 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
@@ -21,7 +21,7 @@ package org.apache.flink.streaming.connectors.flume;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flume.Event;
import org.apache.flume.EventDeliveryException;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
index c296319..02d588c 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
@@ -21,7 +21,7 @@ package org.apache.flink.streaming.connectors.flume;
import java.util.List;
-import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.function.source.RichSourceFunction;
import org.apache.flink.util.Collector;
import org.apache.flume.Context;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
index 414795b..d6e7570 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
@@ -20,8 +20,8 @@
package org.apache.flink.streaming.connectors.flume;
import org.apache.commons.lang.SerializationUtils;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
public class FlumeTopology {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
index 6e18b20..6edfc7a 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
@@ -29,7 +29,7 @@ import kafka.consumer.ConsumerIterator;
import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
-import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.function.source.RichSourceFunction;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
index 295f1cc..55469b2 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
@@ -20,8 +20,8 @@
package org.apache.flink.streaming.connectors.kafka;
import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index fa0be0d..19ea57f 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -23,7 +23,7 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.function.source.RichSourceFunction;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
index 94ae43f..f43740a 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
@@ -20,8 +20,8 @@
package org.apache.flink.streaming.connectors.rabbitmq;
import org.apache.commons.lang.SerializationUtils;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
public class RMQTopology {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
index cb868f5..878a7ad 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
@@ -20,8 +20,8 @@
package org.apache.flink.streaming.connectors.twitter;
import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.examples.function.JSONParseFlatMap;
import org.apache.flink.streaming.examples.wordcount.WordCountCounter;
@@ -73,7 +73,7 @@ public class TwitterLocal {
if (args != null && args.length == 1) {
path = args[0];
} else {
- System.err.println("USAGE:\n haho TwitterLocal itt <pathToPropertiesFile>");
+ System.err.println("USAGE:\nTwitterLocal <pathToPropertiesFile>");
return;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
index 6a464ea..ad0499c 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
@@ -20,8 +20,8 @@
package org.apache.flink.streaming.connectors.twitter;
import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.examples.function.JSONParseFlatMap;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
deleted file mode 100644
index 7aff259..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ /dev/null
@@ -1,975 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang3.SerializationException;
-import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.collector.OutputSelector;
-import org.apache.flink.streaming.api.function.co.CoMapFunction;
-import org.apache.flink.streaming.api.function.co.RichCoMapFunction;
-import org.apache.flink.streaming.api.function.sink.PrintSinkFunction;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
-import org.apache.flink.streaming.api.function.sink.WriteFormatAsText;
-import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByBatches;
-import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByMillis;
-import org.apache.flink.streaming.api.invokable.SinkInvokable;
-import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
-import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
-import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
-import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
-import org.apache.flink.streaming.api.invokable.operator.WindowReduceInvokable;
-import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
-import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
-import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
-import org.apache.flink.streaming.partitioner.DistributePartitioner;
-import org.apache.flink.streaming.partitioner.FieldsPartitioner;
-import org.apache.flink.streaming.partitioner.ForwardPartitioner;
-import org.apache.flink.streaming.partitioner.ShufflePartitioner;
-import org.apache.flink.streaming.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
-
-/**
- * A DataStream represents a stream of elements of the same type. A DataStream
- * can be transformed into another DataStream by applying a transformation as
- * for example
- * <ul>
- * <li>{@link DataStream#map},</li>
- * <li>{@link DataStream#filter}, or</li>
- * <li>{@link DataStream#batchReduce}.</li>
- * </ul>
- *
- * @param <T>
- * The type of the DataStream, i.e., the type of the elements of the
- * DataStream.
- */
-public class DataStream<T> {
-
- protected static Integer counter = 0;
- protected final StreamExecutionEnvironment environment;
- protected String id;
- protected int degreeOfParallelism;
- protected String userDefinedName;
- protected StreamPartitioner<T> partitioner;
- protected List<DataStream<T>> connectedStreams;
-
- protected JobGraphBuilder jobGraphBuilder;
-
- /**
- * Create a new {@link DataStream} in the given execution environment with
- * partitioning set to forward by default.
- *
- * @param environment
- * StreamExecutionEnvironment
- * @param operatorType
- * The type of the operator in the component
- */
- protected DataStream(StreamExecutionEnvironment environment, String operatorType) {
- if (environment == null) {
- throw new NullPointerException("context is null");
- }
-
- // TODO add name based on component number an preferable sequential id
- counter++;
- this.id = operatorType + "-" + counter.toString();
- this.environment = environment;
- this.degreeOfParallelism = environment.getDegreeOfParallelism();
- this.jobGraphBuilder = environment.getJobGraphBuilder();
- this.partitioner = new ForwardPartitioner<T>();
- this.connectedStreams = new ArrayList<DataStream<T>>();
- this.connectedStreams.add(this.copy());
- }
-
- /**
- * Create a new DataStream by creating a copy of another DataStream
- *
- * @param dataStream
- * The DataStream that will be copied.
- */
- protected DataStream(DataStream<T> dataStream) {
- this.environment = dataStream.environment;
- this.id = dataStream.id;
- this.degreeOfParallelism = dataStream.degreeOfParallelism;
- this.userDefinedName = dataStream.userDefinedName;
- this.partitioner = dataStream.partitioner;
- this.jobGraphBuilder = dataStream.jobGraphBuilder;
- this.connectedStreams = new ArrayList<DataStream<T>>();
- for (DataStream<T> stream : dataStream.connectedStreams) {
- this.connectedStreams.add(stream.copy());
- }
-
- }
-
- /**
- * Creates a copy of the DataStream
- *
- * @return The copy
- */
- protected DataStream<T> copy() {
- return new DataStream<T>(this);
- }
-
- /**
- * Partitioning strategy on the stream.
- */
- public static enum ConnectionType {
- SHUFFLE, BROADCAST, FIELD, FORWARD, DISTRIBUTE
- }
-
- /**
- * Returns the ID of the {@link DataStream}.
- *
- * @return ID of the DataStream
- */
- public String getId() {
- return id;
- }
-
- /**
- * Gets the degree of parallelism for this operator.
- *
- * @return The parallelism set for this operator.
- */
- public int getParallelism() {
- return this.degreeOfParallelism;
- }
-
- /**
- * Sets the degree of parallelism for this operator. The degree must be 1 or
- * more.
- *
- * @param dop
- * The degree of parallelism for this operator.
- * @return The operator with set degree of parallelism.
- */
- public DataStream<T> setParallelism(int dop) {
- if (dop < 1) {
- throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
- }
- this.degreeOfParallelism = dop;
-
- jobGraphBuilder.setParallelism(id, degreeOfParallelism);
-
- return this;
- }
-
- /**
- * Sets the mutability of the operator represented by the DataStream. If the
- * operator is set to mutable, the tuples received in the user defined
- * functions, will be reused after the function call. Setting an operator to
- * mutable reduces garbage collection overhead and thus increases
- * scalability.
- *
- * @param isMutable
- * The mutability of the operator.
- * @return The DataStream with mutability set.
- */
- public DataStream<T> setMutability(boolean isMutable) {
- jobGraphBuilder.setMutability(id, isMutable);
- return this;
- }
-
- /**
- * Sets the maximum time frequency (ms) for the flushing of the output
- * buffer. By default the output buffers flush only when they are full.
- *
- * @param timeoutMillis
- * The maximum time between two output flushes.
- * @return The DataStream with buffer timeout set.
- */
- public DataStream<T> setBufferTimeout(long timeoutMillis) {
- jobGraphBuilder.setBufferTimeout(id, timeoutMillis);
- return this;
- }
-
- /**
- * Connecting {@link DataStream} outputs with each other for applying joint
- * operators on them. The DataStreams connected using this operator will be
- * transformed simultaneously. It creates a joint output of the connected
- * DataStreams.
- *
- * @param streams
- * The DataStreams to connect output with.
- * @return The connected DataStream.
- */
- public DataStream<T> connectWith(DataStream<T>... streams) {
- DataStream<T> returnStream = this.copy();
-
- for (DataStream<T> stream : streams) {
- addConnection(returnStream, stream);
- }
- return returnStream;
- }
-
- /**
- * Operator used for directing tuples to specific named outputs. Sets an
- * {@link OutputSelector} for the vertex. The tuples emitted from this
- * vertex will be sent to the output names selected by the OutputSelector.
- * Unnamed outputs will not receive any tuples.
- *
- * @param outputSelector
- * The user defined OutputSelector for directing the tuples.
- * @return The {@link SplitDataStream}
- */
- public SplitDataStream<T> split(OutputSelector<T> outputSelector) {
- try {
- for (DataStream<T> stream : connectedStreams) {
- jobGraphBuilder.setOutputSelector(stream.id,
- SerializationUtils.serialize(outputSelector));
- }
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize OutputSelector");
- }
-
- return new SplitDataStream<T>(this);
- }
-
- /**
- * Sets the partitioning of the {@link DataStream} so that the output tuples
- * are partitioned by their hashcode and are sent to only one component.
- *
- * @param keyposition
- * The field used to compute the hashcode.
- * @return The DataStream with field partitioning set.
- */
- public DataStream<T> partitionBy(int keyposition) {
- if (keyposition < 0) {
- throw new IllegalArgumentException("The position of the field must be non-negative");
- }
-
- return setConnectionType(new FieldsPartitioner<T>(keyposition));
- }
-
- /**
- * Sets the partitioning of the {@link DataStream} so that the output tuples
- * are broadcasted to every parallel instance of the next component.
- *
- * @return The DataStream with broadcast partitioning set.
- */
- public DataStream<T> broadcast() {
- return setConnectionType(new BroadcastPartitioner<T>());
- }
-
- /**
- * Sets the partitioning of the {@link DataStream} so that the output tuples
- * are shuffled to the next component.
- *
- * @return The DataStream with shuffle partitioning set.
- */
- public DataStream<T> shuffle() {
- return setConnectionType(new ShufflePartitioner<T>());
- }
-
- /**
- * Sets the partitioning of the {@link DataStream} so that the output tuples
- * are forwarded to the local subtask of the next component. This is the
- * default partitioner setting.
- *
- * @return The DataStream with shuffle partitioning set.
- */
- public DataStream<T> forward() {
- return setConnectionType(new ForwardPartitioner<T>());
- }
-
- /**
- * Sets the partitioning of the {@link DataStream} so that the output tuples
- * are distributed evenly to the next component.
- *
- * @return The DataStream with shuffle partitioning set.
- */
- public DataStream<T> distribute() {
- return setConnectionType(new DistributePartitioner<T>());
- }
-
- /**
- * Applies a Map transformation on a {@link DataStream}. The transformation
- * calls a {@link MapFunction} for each element of the DataStream. Each
- * MapFunction call returns exactly one element. The user can also extend
- * {@link RichMapFunction} to gain access to other features provided by the
- * {@link RichFuntion} interface.
- *
- * @param mapper
- * The MapFunction that is called for each element of the
- * DataStream.
- * @param <R>
- * output type
- * @return The transformed DataStream.
- */
- public <R> StreamOperator<R> map(MapFunction<T, R> mapper) {
- return addFunction("map", mapper, new FunctionTypeWrapper<T, Tuple, R>(mapper,
- MapFunction.class, 0, -1, 1), new MapInvokable<T, R>(mapper));
- }
-
- /**
- * Applies a FlatMap transformation on a {@link DataStream}. The
- * transformation calls a {@link FlatMapFunction} for each element of the
- * DataStream. Each FlatMapFunction call can return any number of elements
- * including none. The user can also extend {@link RichFlatMapFunction} to
- * gain access to other features provided by the {@link RichFuntion}
- * interface.
- *
- * @param flatMapper
- * The FlatMapFunction that is called for each element of the
- * DataStream
- *
- * @param <R>
- * output type
- * @return The transformed DataStream.
- */
- public <R> StreamOperator<R> flatMap(FlatMapFunction<T, R> flatMapper) {
- return addFunction("flatMap", flatMapper, new FunctionTypeWrapper<T, Tuple, R>(flatMapper,
- FlatMapFunction.class, 0, -1, 1), new FlatMapInvokable<T, R>(flatMapper));
- }
-
- /**
- * Applies a CoMap transformation on two separate {@link DataStream}s. The
- * transformation calls a {@link CoMapFunction#map1(Tuple)} for each element
- * of the first DataStream (on which .coMapWith was called) and
- * {@link CoMapFunction#map2(Tuple)} for each element of the second
- * DataStream. Each CoMapFunction call returns exactly one element. The user
- * can also extend {@link RichCoMapFunction} to gain access to other
- * features provided by the {@link RichFuntion} interface.
- *
- * @param coMapper
- * The CoMapFunction used to jointly transform the two input
- * DataStreams
- * @param otherStream
- * The DataStream that will be transformed with
- * {@link CoMapFunction#map2(Tuple)}
- * @return The transformed DataStream
- */
- public <T2, R> StreamOperator<R> coMapWith(CoMapFunction<T, T2, R> coMapper,
- DataStream<T2> otherStream) {
- return addCoFunction("coMap", this.copy(), otherStream.copy(), coMapper,
- new FunctionTypeWrapper<T, T2, R>(coMapper, CoMapFunction.class, 0, 1, 2),
- new CoMapInvokable<T, T2, R>(coMapper));
- }
-
- /**
- * Applies a reduce transformation on preset chunks of the DataStream. The
- * transformation calls a {@link GroupReduceFunction} for each tuple batch
- * of the predefined size. Each GroupReduceFunction call can return any
- * number of elements including none. The user can also extend
- * {@link RichGroupReduceFunction} to gain access to other features provided
- * by the {@link RichFuntion} interface.
- *
- *
- * @param reducer
- * The GroupReduceFunction that is called for each tuple batch.
- * @param batchSize
- * The number of tuples grouped together in the batch.
- * @param <R>
- * output type
- * @return The modified DataStream.
- */
- public <R> StreamOperator<R> batchReduce(GroupReduceFunction<T, R> reducer, int batchSize) {
- return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
- GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<T, R>(reducer,
- batchSize));
- }
-
- /**
- * Applies a reduce transformation on preset "time" chunks of the
- * DataStream. The transformation calls a {@link GroupReduceFunction} on
- * records received during the predefined time window. The window shifted
- * after each reduce call. Each GroupReduceFunction call can return any
- * number of elements including none.The user can also extend
- * {@link RichGroupReduceFunction} to gain access to other features provided
- * by the {@link RichFuntion} interface.
- *
- *
- * @param reducer
- * The GroupReduceFunction that is called for each time window.
- * @param windowSize
- * The time window to run the reducer on, in milliseconds.
- * @param <R>
- * output type
- * @return The modified DataStream.
- */
- public <R> StreamOperator<R> windowReduce(GroupReduceFunction<T, R> reducer, long windowSize) {
- return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
- GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<T, R>(reducer,
- windowSize));
- }
-
- /**
- * Applies a Filter transformation on a {@link DataStream}. The
- * transformation calls a {@link FilterFunction} for each element of the
- * DataStream and retains only those element for which the function returns
- * true. Elements for which the function returns false are filtered. The
- * user can also extend {@link RichFilterFunction} to gain access to other
- * features provided by the {@link RichFuntion} interface.
- *
- * @param filter
- * The FilterFunction that is called for each element of the
- * DataSet.
- * @return The filtered DataStream.
- */
- public StreamOperator<T> filter(FilterFunction<T> filter) {
- return addFunction("filter", filter, new FunctionTypeWrapper<T, Tuple, T>(filter,
- FilterFunction.class, 0, -1, 0), new FilterInvokable<T>(filter));
- }
-
- /**
- * Writes a DataStream to the standard output stream (stdout). For each
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @return The closed DataStream.
- */
- public DataStream<T> print() {
- DataStream<T> inputStream = this.copy();
- PrintSinkFunction<T> printFunction = new PrintSinkFunction<T>();
- DataStream<T> returnStream = addSink(inputStream, printFunction, null);
-
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
-
- return returnStream;
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsText(String path) {
- return writeAsText(this, path, new WriteFormatAsText<T>(), 1, null);
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically, in every millis milliseconds. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param millis
- * is the file update frequency
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsText(String path, long millis) {
- return writeAsText(this, path, new WriteFormatAsText<T>(), millis, null);
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically in equally sized batches. For every
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param batchSize
- * is the size of the batches, i.e. the number of tuples written
- * to the file at a time
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsText(String path, int batchSize) {
- return writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, null);
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically, in every millis milliseconds. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param millis
- * is the file update frequency
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * system time.
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsText(String path, long millis, T endTuple) {
- return writeAsText(this, path, new WriteFormatAsText<T>(), millis, endTuple);
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically in equally sized batches. For every
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param batchSize
- * is the size of the batches, i.e. the number of tuples written
- * to the file at a time
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * batchSize.
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsText(String path, int batchSize, T endTuple) {
- return writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, endTuple);
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically, in every millis milliseconds. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param millis
- * is the file update frequency
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * system time.
- *
- * @return the data stream constructed
- */
- private DataStream<T> writeAsText(DataStream<T> inputStream, String path,
- WriteFormatAsText<T> format, long millis, T endTuple) {
- DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<T>(path,
- format, millis, endTuple), null);
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
- jobGraphBuilder.setMutability(returnStream.getId(), false);
- return returnStream;
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically in equally sized batches. For every
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param batchSize
- * is the size of the batches, i.e. the number of tuples written
- * to the file at a time
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * batchSize.
- *
- * @return the data stream constructed
- */
- private DataStream<T> writeAsText(DataStream<T> inputStream, String path,
- WriteFormatAsText<T> format, int batchSize, T endTuple) {
- DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<T>(path,
- format, batchSize, endTuple), null);
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
- jobGraphBuilder.setMutability(returnStream.getId(), false);
- return returnStream;
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsCsv(String path) {
- return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), 1, null);
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically, in every millis milliseconds. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param millis
- * is the file update frequency
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsCsv(String path, long millis) {
- return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, null);
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically in equally sized batches. For every
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param batchSize
- * is the size of the batches, i.e. the number of tuples written
- * to the file at a time
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsCsv(String path, int batchSize) {
- return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, null);
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically, in every millis milliseconds. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param millis
- * is the file update frequency
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * system time.
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsCsv(String path, long millis, T endTuple) {
- return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, endTuple);
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically in equally sized batches. For every
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param batchSize
- * is the size of the batches, i.e. the number of tuples written
- * to the file at a time
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * batchSize.
- *
- * @return The closed DataStream
- */
- public DataStream<T> writeAsCsv(String path, int batchSize, T endTuple) {
- setMutability(false);
- return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, endTuple);
- }
-
- /**
- * Writes a DataStream to the file specified by path in csv format. The
- * writing is performed periodically, in every millis milliseconds. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param millis
- * is the file update frequency
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * system time.
- *
- * @return the data stream constructed
- */
- private DataStream<T> writeAsCsv(DataStream<T> inputStream, String path,
- WriteFormatAsCsv<T> format, long millis, T endTuple) {
- DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<T>(path,
- format, millis, endTuple));
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
- jobGraphBuilder.setMutability(returnStream.getId(), false);
- return returnStream;
- }
-
- /**
- * Writes a DataStream to the file specified by path in csv format. The
- * writing is performed periodically in equally sized batches. For every
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param batchSize
- * is the size of the batches, i.e. the number of tuples written
- * to the file at a time
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * batchSize.
- *
- * @return the data stream constructed
- */
- private DataStream<T> writeAsCsv(DataStream<T> inputStream, String path,
- WriteFormatAsCsv<T> format, int batchSize, T endTuple) {
- DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<T>(path,
- format, batchSize, endTuple), null);
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
- jobGraphBuilder.setMutability(returnStream.getId(), false);
- return returnStream;
- }
-
- /**
- * Initiates an iterative part of the program that executes multiple times
- * and feeds back data streams. The iterative part needs to be closed by
- * calling {@link IterativeDataStream#closeWith(DataStream)}. The
- * transformation of this IterativeDataStream will be the iteration head.
- * The data stream given to the {@code closeWith(DataStream)} method is the
- * data stream that will be fed back and used as the input for the iteration
- * head. Unlike in batch processing by default the output of the iteration
- * stream is directed to both to the iteration head and the next component.
- * To direct tuples to the iteration head or the output specifically one can
- * use the {@code split(OutputSelector)} on the iteration tail while
- * referencing the iteration head as 'iterate'.
- *
- * The iteration edge will be partitioned the same way as the first input of
- * the iteration head.
- *
- * @return The iterative data stream created.
- */
- public IterativeDataStream<T> iterate() {
- return new IterativeDataStream<T>(this);
- }
-
- protected <R> DataStream<T> addIterationSource(String iterationID) {
- DataStream<R> returnStream = new DataStream<R>(environment, "iterationSource");
-
- jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
- degreeOfParallelism);
-
- return this.copy();
- }
-
- /**
- * Internal function for passing the user defined functions to the JobGraph
- * of the job.
- *
- * @param functionName
- * name of the function
- * @param function
- * the user defined function
- * @param functionInvokable
- * the wrapping JobVertex instance
- * @param <R>
- * type of the return stream
- * @return the data stream constructed
- */
- private <R> StreamOperator<R> addFunction(String functionName, final Function function,
- TypeSerializerWrapper<T, Tuple, R> typeWrapper,
- UserTaskInvokable<T, R> functionInvokable) {
-
- DataStream<T> inputStream = this.copy();
- StreamOperator<R> returnStream = new SingleInputStreamOperator<T, R>(environment,
- functionName);
-
- try {
- jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, typeWrapper,
- functionName, SerializationUtils.serialize((Serializable) function),
- degreeOfParallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize user defined function");
- }
-
- connectGraph(inputStream, returnStream.getId(), 0);
-
- if (inputStream instanceof IterativeDataStream) {
- returnStream.addIterationSource(((IterativeDataStream<T>) inputStream).iterationID
- .toString());
- }
-
- if (inputStream instanceof NamedDataStream) {
- returnStream.name(inputStream.userDefinedName);
- }
-
- return returnStream;
- }
-
- protected <T1, T2, R> StreamOperator<R> addCoFunction(String functionName,
- DataStream<T1> inputStream1, DataStream<T2> inputStream2, final Function function,
- TypeSerializerWrapper<T1, T2, R> typeWrapper, CoInvokable<T1, T2, R> functionInvokable) {
-
- StreamOperator<R> returnStream = new TwoInputStreamOperator<T1, T2, R>(environment,
- functionName);
-
- try {
- jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, typeWrapper,
- functionName, SerializationUtils.serialize((Serializable) function),
- degreeOfParallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize user defined function");
- }
-
- connectGraph(inputStream1, returnStream.getId(), 1);
- connectGraph(inputStream2, returnStream.getId(), 2);
-
- if ((inputStream1 instanceof NamedDataStream) && (inputStream2 instanceof NamedDataStream)) {
- throw new RuntimeException("An operator cannot have two names");
- } else {
- if (inputStream1 instanceof NamedDataStream) {
- returnStream.name(inputStream1.userDefinedName);
- }
-
- if (inputStream2 instanceof NamedDataStream) {
- returnStream.name(inputStream2.userDefinedName);
- }
- }
- // TODO consider iteration
-
- return returnStream;
- }
-
- /**
- * Gives the data transformation(vertex) a user defined name in order to use
- * with directed outputs. The {@link OutputSelector} of the input vertex
- * should use this name for directed emits.
- *
- * @param name
- * The name to set
- * @return The named DataStream.
- */
- protected DataStream<T> name(String name) {
- // TODO copy DataStream?
- if (name == "") {
- throw new IllegalArgumentException("User defined name must not be empty string");
- }
-
- userDefinedName = name;
- jobGraphBuilder.setUserDefinedName(id, name);
-
- return this;
- }
-
- /**
- * Connects two DataStreams
- *
- * @param returnStream
- * The other DataStream will connected to this
- * @param stream
- * This DataStream will be connected to returnStream
- */
- private void addConnection(DataStream<T> returnStream, DataStream<T> stream) {
- if ((stream instanceof NamedDataStream) || (returnStream instanceof NamedDataStream)) {
- if (!returnStream.userDefinedName.equals(stream.userDefinedName)) {
- throw new RuntimeException("Error: Connected NamedDataStreams must have same names");
- }
- }
- returnStream.connectedStreams.add(stream.copy());
- }
-
- private DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
- DataStream<T> returnStream = this.copy();
-
- for (DataStream<T> stream : returnStream.connectedStreams) {
- stream.partitioner = partitioner;
- }
-
- return returnStream;
- }
-
- /**
- * Internal function for assembling the underlying
- * {@link org.apache.flink.nephele.jobgraph.JobGraph} of the job. Connects
- * the outputs of the given input stream to the specified output stream
- * given by the outputID.
- *
- * @param inputStream
- * input data stream
- * @param outputID
- * ID of the output
- * @param typeNumber
- * Number of the type (used at co-functions)
- */
- private <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
- for (DataStream<X> stream : inputStream.connectedStreams) {
- jobGraphBuilder.setEdge(stream.getId(), outputID, stream.partitioner, typeNumber);
- }
- }
-
- /**
- * Adds the given sink to this DataStream. Only streams with sinks added
- * will be executed once the {@link StreamExecutionEnvironment#execute()}
- * method is called.
- *
- * @param sinkFunction
- * The object containing the sink's invoke function.
- * @return The closed DataStream.
- */
- public DataStream<T> addSink(SinkFunction<T> sinkFunction) {
- return addSink(this.copy(), sinkFunction);
- }
-
- private DataStream<T> addSink(DataStream<T> inputStream, SinkFunction<T> sinkFunction) {
- return addSink(inputStream, sinkFunction, new FunctionTypeWrapper<T, Tuple, T>(
- sinkFunction, SinkFunction.class, 0, -1, 0));
- }
-
- private DataStream<T> addSink(DataStream<T> inputStream, SinkFunction<T> sinkFunction,
- TypeSerializerWrapper<T, Tuple, T> typeWrapper) {
- DataStream<T> returnStream = new DataStream<T>(environment, "sink");
-
- try {
- jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<T>(sinkFunction),
- typeWrapper, "sink", SerializationUtils.serialize(sinkFunction),
- degreeOfParallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize SinkFunction");
- }
-
- inputStream.connectGraph(inputStream, returnStream.getId(), 0);
-
- if (this.copy() instanceof NamedDataStream) {
- returnStream.name(inputStream.userDefinedName);
- }
-
- return returnStream;
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
deleted file mode 100644
index 71e129c..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-import org.apache.flink.streaming.partitioner.ForwardPartitioner;
-
-/**
- * The iterative data stream represents the start of an iteration in a
- * {@link DataStream}.
- *
- * @param <T>
- * Type of the DataStream
- */
-public class IterativeDataStream<T> extends SingleInputStreamOperator<T, T> {
-
- static Integer iterationCount = 0;
- protected Integer iterationID;
-
- protected IterativeDataStream(DataStream<T> dataStream) {
- super(dataStream);
- iterationID = iterationCount;
- iterationCount++;
- }
-
- protected IterativeDataStream(DataStream<T> dataStream, Integer iterationID) {
- super(dataStream);
- this.iterationID = iterationID;
- }
-
- /**
- * Closes the iteration. This method defines the end of the iterative
- * program part. By default the DataStream represented by the parameter will
- * be fed back to the iteration head, however the user can explicitly select
- * which tuples should be iterated by {@code directTo(OutputSelector)}.
- * Tuples directed to 'iterate' will be fed back to the iteration head.
- *
- * @param iterationResult
- * The data stream that can be fed back to the next iteration.
- *
- */
- public DataStream<T> closeWith(DataStream<T> iterationResult) {
- return closeWith(iterationResult, null);
- }
-
- /**
- * Closes the iteration. This method defines the end of the iterative
- * program part. By default the DataStream represented by the parameter will
- * be fed back to the iteration head, however the user can explicitly select
- * which tuples should be iterated by {@code directTo(OutputSelector)}.
- * Tuples directed to 'iterate' will be fed back to the iteration head.
- *
- * @param iterationResult
- * The data stream that can be fed back to the next iteration.
- * @param iterationName
- * Name of the iteration edge (backward edge to iteration head)
- * when used with directed emits
- *
- */
- public <R> DataStream<T> closeWith(DataStream<T> iterationResult, String iterationName) {
- DataStream<R> returnStream = new DataStream<R>(environment, "iterationSink");
-
- jobGraphBuilder.addIterationSink(returnStream.getId(), iterationResult.getId(),
- iterationID.toString(), iterationResult.getParallelism(), iterationName);
-
- jobGraphBuilder.setIterationSourceParallelism(iterationID.toString(),
- iterationResult.getParallelism());
-
- for (DataStream<T> stream : iterationResult.connectedStreams) {
- String inputID = stream.getId();
- jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<T>(), 0);
- }
-
- return iterationResult;
- }
-
- @Override
- protected DataStream<T> copy() {
- return new IterativeDataStream<T>(this, iterationID);
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/LocalStreamEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/LocalStreamEnvironment.java
deleted file mode 100755
index d62d003..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/LocalStreamEnvironment.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-import org.apache.flink.streaming.util.ClusterUtil;
-
-public class LocalStreamEnvironment extends StreamExecutionEnvironment {
-
- /**
- * Executes the JobGraph of the on a mini cluster of CLusterUtil.
- *
- */
- @Override
- public void execute() {
- ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(), getExecutionParallelism());
- }
-
- public void executeTest(long memorySize) {
- ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(), getExecutionParallelism(),
- memorySize);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/NamedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/NamedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/NamedDataStream.java
deleted file mode 100755
index 1edfa6f..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/NamedDataStream.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-public class NamedDataStream<T> extends DataStream<T> {
-
- protected NamedDataStream(DataStream<T> dataStream) {
- super(dataStream);
- }
-
- @Override
- protected DataStream<T> copy() {
- return new NamedDataStream<T>(this);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/RemoteStreamEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/RemoteStreamEnvironment.java
deleted file mode 100644
index ec4a2b7..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/RemoteStreamEnvironment.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.flink.client.program.Client;
-import org.apache.flink.client.program.JobWithJars;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-
-public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
- private static final Log log = LogFactory.getLog(RemoteStreamEnvironment.class);
-
- private String host;
- private int port;
- private String[] jarFiles;
-
- /**
- * Creates a new RemoteStreamEnvironment that points to the master
- * (JobManager) described by the given host name and port.
- *
- * @param host
- * The host name or address of the master (JobManager), where the
- * program should be executed.
- * @param port
- * The port of the master (JobManager), where the program should
- * be executed.
- * @param jarFiles
- * The JAR files with code that needs to be shipped to the
- * cluster. If the program uses user-defined functions,
- * user-defined input formats, or any libraries, those must be
- * provided in the JAR files.
- */
- public RemoteStreamEnvironment(String host, int port, String... jarFiles) {
- if (host == null) {
- throw new NullPointerException("Host must not be null.");
- }
-
- if (port < 1 || port >= 0xffff) {
- throw new IllegalArgumentException("Port out of range");
- }
-
- this.host = host;
- this.port = port;
- this.jarFiles = jarFiles;
- }
-
- @Override
- public void execute() {
- if (log.isInfoEnabled()) {
- log.info("Running remotely at " + host + ":" + port);
- }
-
- JobGraph jobGraph = jobGraphBuilder.getJobGraph();
-
- for (int i = 0; i < jarFiles.length; i++) {
- File file = new File(jarFiles[i]);
- try {
- JobWithJars.checkJarFile(file);
- } catch (IOException e) {
- throw new RuntimeException("Problem with jar file " + file.getAbsolutePath(), e);
- }
- jobGraph.addJar(new Path(file.getAbsolutePath()));
- }
-
- Configuration configuration = jobGraph.getJobConfiguration();
- Client client = new Client(new InetSocketAddress(host, port), configuration);
-
- try {
- client.run(jobGraph, true);
- } catch (ProgramInvocationException e) {
- throw new RuntimeException("Cannot execute job due to ProgramInvocationException", e);
- }
- }
-
- @Override
- public String toString() {
- return "Remote Environment (" + this.host + ":" + this.port + " - DOP = "
- + (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) + ")";
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SingleInputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SingleInputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SingleInputStreamOperator.java
deleted file mode 100755
index 43dfe36..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SingleInputStreamOperator.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-/**
- * The SingleInputStreamOperator represents a {@link DataStream} transformed
- * with some user defined operator.
- *
- * @param <IN>
- * Input type of the operator.
- * @param <OUT>
- * Output Type of the operator.
- */
-public class SingleInputStreamOperator<IN, OUT> extends StreamOperator<OUT> {
-
- protected SingleInputStreamOperator(StreamExecutionEnvironment environment, String operatorType) {
- super(environment, operatorType);
- }
-
- protected SingleInputStreamOperator(DataStream<OUT> dataStream) {
- super(dataStream);
- }
-
- @Override
- protected DataStream<OUT> copy() {
- return new SingleInputStreamOperator<IN, OUT>(this);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
deleted file mode 100755
index 627aa42..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-public class SplitDataStream<T> extends DataStream<T> {
-
- protected SplitDataStream(DataStream<T> dataStream) {
- super(dataStream);
- }
-
- /**
- * Sets the output name for which the vertex will receive tuples from the
- * preceding Directed stream
- *
- * @param outputName
- * The output name for which the operator will receive the input.
- * @return Returns the modified DataStream
- */
- public NamedDataStream<T> select(String outputName) {
- NamedDataStream<T> returnStream = new NamedDataStream<T>(this);
- returnStream.userDefinedName = outputName;
- return returnStream;
- }
-
- @Override
- protected DataStream<T> copy() {
- return new SplitDataStream<T>(this);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index 4572a84..7cfc808 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -20,6 +20,8 @@
package org.apache.flink.streaming.api;
import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
import org.apache.commons.lang3.SerializationException;
import org.apache.commons.lang3.SerializationUtils;
@@ -164,9 +166,9 @@ public class StreamConfig {
return config.getString(FUNCTION_NAME, "");
}
- public void setUserDefinedName(String userDefinedName) {
- if (userDefinedName != null) {
- config.setString(USER_DEFINED_NAME, userDefinedName);
+ public void setUserDefinedName(List<String> userDefinedName) {
+ if (!userDefinedName.isEmpty()) {
+ config.setBytes(USER_DEFINED_NAME, SerializationUtils.serialize((Serializable) userDefinedName));
}
}
@@ -223,14 +225,15 @@ public class StreamConfig {
SerializationUtils.serialize(new ShufflePartitioner<T>())));
}
- public void setOutputName(int outputIndex, String outputName) {
+ public void setOutputName(int outputIndex, List<String> outputName) {
if (outputName != null) {
- config.setString(OUTPUT_NAME + outputIndex, outputName);
+ config.setBytes(OUTPUT_NAME + outputIndex, SerializationUtils.serialize((Serializable) outputName));
}
}
- public String getOutputName(int outputIndex) {
- return config.getString(OUTPUT_NAME + outputIndex, null);
+ @SuppressWarnings("unchecked")
+ public List<String> getOutputName(int outputIndex) {
+ return (List<String>) SerializationUtils.deserialize(config.getBytes(OUTPUT_NAME + outputIndex, null));
}
public void setNumberOfInputs(int numberOfInputs) {
[23/51] [abbrv] [streaming] Wrapped serializers to make component
construction simpler
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
index 5532626..911b550 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
@@ -29,7 +29,8 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.util.MutableObjectIterator;
import org.apache.flink.util.StringUtils;
-public class StreamIterationSink<IN extends Tuple> extends SingleInputAbstractStreamComponent<IN, IN> {
+public class StreamIterationSink<IN extends Tuple> extends
+ SingleInputAbstractStreamComponent<IN, IN> {
private static final Log LOG = LogFactory.getLog(StreamIterationSink.class);
@@ -51,13 +52,13 @@ public class StreamIterationSink<IN extends Tuple> extends SingleInputAbstractSt
iterationId = configuration.getIterationId();
dataChannel = BlockingQueueBroker.instance().get(iterationId);
-
+
} catch (Exception e) {
throw new StreamComponentException(String.format(
"Cannot register inputs of StreamIterationSink %s", iterationId), e);
}
}
-
+
@Override
public void invoke() throws Exception {
if (LOG.isDebugEnabled()) {
@@ -93,6 +94,5 @@ public class StreamIterationSink<IN extends Tuple> extends SingleInputAbstractSt
@Override
protected void setInvokable() {
-
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
index d020058..cf3d47e 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
@@ -32,7 +32,8 @@ import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-public class StreamIterationSource<OUT extends Tuple> extends SingleInputAbstractStreamComponent<Tuple, OUT> {
+public class StreamIterationSource<OUT extends Tuple> extends
+ SingleInputAbstractStreamComponent<Tuple, OUT> {
private static final Log LOG = LogFactory.getLog(StreamIterationSource.class);
@@ -68,7 +69,7 @@ public class StreamIterationSource<OUT extends Tuple> extends SingleInputAbstrac
}
}
-
+
@Override
public void invoke() throws Exception {
if (LOG.isDebugEnabled()) {
@@ -100,5 +101,4 @@ public class StreamIterationSource<OUT extends Tuple> extends SingleInputAbstrac
@Override
protected void setInvokable() {
}
-
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
index 7ac117e..0ead3c6 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
@@ -28,10 +28,10 @@ public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamCompo
private static final Log LOG = LogFactory.getLog(StreamSink.class);
- private StreamRecordInvokable<IN, IN> userFunction;
+ private StreamRecordInvokable<IN, IN> userInvokable;
public StreamSink() {
- userFunction = null;
+ userInvokable = null;
}
@Override
@@ -39,18 +39,18 @@ public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamCompo
try {
setConfigInputs();
setSinkSerializer();
-
+
inputIter = createInputIterator(inputs, inTupleSerializer);
} catch (Exception e) {
throw new StreamComponentException("Cannot register inputs for "
+ getClass().getSimpleName(), e);
}
}
-
+
@Override
- protected void setInvokable() {
- userFunction = getInvokable();
- userFunction.initialize(collector, inputIter, inTupleSerializer, isMutable);
+ protected void setInvokable() {
+ userInvokable = getInvokable();
+ userInvokable.initialize(collector, inputIter, inTupleSerializer, isMutable);
}
@Override
@@ -59,7 +59,7 @@ public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamCompo
LOG.debug("SINK " + name + " invoked");
}
- userFunction.invoke();
+ userInvokable.invoke();
if (LOG.isDebugEnabled()) {
LOG.debug("SINK " + name + " invoke finished");
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
index 856c917..70b8242 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
@@ -27,7 +27,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
+import org.apache.flink.streaming.api.invokable.SourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamComponent<Tuple, OUT> {
@@ -35,13 +35,13 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
private static final Log LOG = LogFactory.getLog(StreamSource.class);
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
- private UserSourceInvokable<OUT> userFunction;
+ private SourceInvokable<OUT> userInvokable;
private static int numSources;
public StreamSource() {
outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
- userFunction = null;
+ userInvokable = null;
numSources = newComponent();
instanceID = numSources;
}
@@ -53,13 +53,14 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
} catch (StreamComponentException e) {
throw new StreamComponentException("Cannot register outputs for "
+ getClass().getSimpleName(), e);
- }
+ }
}
-
+
@Override
protected void setInvokable() {
- // Default value is a TaskInvokable even if it was called from a source
- userFunction = getInvokable();
+ userInvokable = getInvokable();
+ // setCollector();
+ userInvokable.setCollector(collector);
}
@Override
@@ -72,7 +73,7 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
output.initializeSerializers();
}
- userFunction.invoke(collector);
+ userInvokable.invoke();
if (LOG.isDebugEnabled()) {
LOG.debug("SOURCE " + name + " invoke finished with instance id " + instanceID);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
index 19b1c4b..7cb1d71 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
@@ -36,29 +36,28 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
private static final Log LOG = LogFactory.getLog(StreamTask.class);
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
- private StreamRecordInvokable<IN, OUT> userFunction;
+ private StreamRecordInvokable<IN, OUT> userInvokable;
private static int numTasks;
public StreamTask() {
outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
- userFunction = null;
+ userInvokable = null;
numTasks = newComponent();
instanceID = numTasks;
}
-
+
@Override
public void setInputsOutputs() {
setConfigInputs();
setConfigOutputs(outputs);
- inputIter = createInputIterator(inputs, inTupleSerializer);
+ inputIter = createInputIterator(inputs, inTupleSerializer);
}
-
+
@Override
protected void setInvokable() {
- // Default value is a TaskInvokable even if it was called from a source
- userFunction = getInvokable();
- userFunction.initialize(collector, inputIter, inTupleSerializer, isMutable);
+ userInvokable = getInvokable();
+ userInvokable.initialize(collector, inputIter, inTupleSerializer, isMutable);
}
@Override
@@ -71,7 +70,7 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
output.initializeSerializers();
}
- userFunction.invoke();
+ userInvokable.invoke();
if (LOG.isDebugEnabled()) {
LOG.debug("TASK " + name + " invoke finished with instance id " + instanceID);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
new file mode 100644
index 0000000..c868e4f
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
@@ -0,0 +1,74 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.util.serialization;
+
+import java.io.IOException;
+
+import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+public class FunctionTypeWrapper<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
+ TypeSerializerWrapper<IN1, IN2, OUT> {
+ private static final long serialVersionUID = 1L;
+
+ private AbstractFunction function;
+ private Class<? extends AbstractFunction> functionSuperClass;
+ private int inTypeParameter1;
+ private int inTypeParameter2;
+ private int outTypeParameter;
+
+ public FunctionTypeWrapper(AbstractFunction function,
+ Class<? extends AbstractFunction> functionSuperClass, int inTypeParameter1,
+ int inTypeParameter2, int outTypeParameter) {
+ this.function = function;
+ this.functionSuperClass = functionSuperClass;
+ this.inTypeParameter1 = inTypeParameter1;
+ this.inTypeParameter2 = inTypeParameter2;
+ this.outTypeParameter = outTypeParameter;
+ setTupleTypeInfo();
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws IOException,
+ ClassNotFoundException {
+ in.defaultReadObject();
+ setTupleTypeInfo();
+ }
+
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ @Override
+ protected void setTupleTypeInfo() {
+ if (inTypeParameter1 != -1) {
+ inTupleTypeInfo1 = (TupleTypeInfo) TypeExtractor.createTypeInfo(functionSuperClass,
+ function.getClass(), inTypeParameter1, null, null);
+ }
+
+ if (inTypeParameter2 != -1) {
+ inTupleTypeInfo2 = (TupleTypeInfo) TypeExtractor.createTypeInfo(functionSuperClass,
+ function.getClass(), inTypeParameter2, null, null);
+ }
+
+ if (outTypeParameter != -1) {
+ outTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(functionSuperClass,
+ function.getClass(), outTypeParameter, null, null);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
new file mode 100644
index 0000000..5ed2312
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
@@ -0,0 +1,61 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.util.serialization;
+
+import java.io.IOException;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+public class ObjectTypeWrapper<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
+ TypeSerializerWrapper<IN1, IN2, OUT> {
+ private static final long serialVersionUID = 1L;
+
+ private Object inInstance1;
+ private Object inInstance2;
+ private Object outInstance;
+
+ public ObjectTypeWrapper(Object inInstance1, Object inInstance2, Object outInstance) {
+ this.inInstance1 = inInstance1;
+ this.inInstance2 = inInstance2;
+ this.outInstance = outInstance;
+ setTupleTypeInfo();
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws IOException,
+ ClassNotFoundException {
+ in.defaultReadObject();
+ setTupleTypeInfo();
+ }
+
+ @Override
+ protected void setTupleTypeInfo() {
+ if (inInstance1 != null) {
+ inTupleTypeInfo1 = new TupleTypeInfo<IN1>(TypeExtractor.getForObject(inInstance1));
+ }
+ if (inInstance2 != null) {
+ inTupleTypeInfo2 = new TupleTypeInfo<IN2>(TypeExtractor.getForObject(inInstance2));
+ }
+ if (outInstance != null) {
+ outTupleTypeInfo = new TupleTypeInfo<OUT>(TypeExtractor.getForObject(outInstance));
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
new file mode 100644
index 0000000..473ce7c
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
@@ -0,0 +1,57 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.util.serialization;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
+public abstract class TypeSerializerWrapper<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple>
+ implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ protected transient TupleTypeInfo<IN1> inTupleTypeInfo1 = null;
+ protected transient TupleTypeInfo<IN2> inTupleTypeInfo2 = null;
+ protected transient TupleTypeInfo<OUT> outTupleTypeInfo = null;
+
+ public TupleTypeInfo<IN1> getInputTupleTypeInfo1() {
+ if (inTupleTypeInfo1 == null) {
+ throw new RuntimeException("There is no TypeInfo for the first input");
+ }
+ return inTupleTypeInfo1;
+ }
+
+ public TupleTypeInfo<IN2> getInputTupleTypeInfo2() {
+ if (inTupleTypeInfo1 == null) {
+ throw new RuntimeException("There is no TypeInfo for the first input");
+ }
+ return inTupleTypeInfo2;
+ }
+
+ public TupleTypeInfo<OUT> getOutputTupleTypeInfo() {
+ if (inTupleTypeInfo1 == null) {
+ throw new RuntimeException("There is no TypeInfo for the first input");
+ }
+ return outTupleTypeInfo;
+ }
+
+ protected abstract void setTupleTypeInfo();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
index fc0efcf..0f22262 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
@@ -19,7 +19,7 @@
package org.apache.flink.streaming.api;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
import java.io.BufferedReader;
import java.io.File;
@@ -30,7 +30,9 @@ import java.util.List;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.streaming.util.LogUtils;
import org.apache.flink.util.Collector;
+import org.apache.log4j.Level;
import org.junit.Test;
public class WriteAsTextTest {
@@ -110,9 +112,11 @@ public class WriteAsTextTest {
@Test
public void test() throws Exception {
-
+
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+ LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+
@SuppressWarnings("unused")
DataStream<Tuple1<Integer>> dataStream1 = env.addSource(new MySource1(), 1).writeAsText(
"test1.txt");
@@ -151,11 +155,11 @@ public class WriteAsTextTest {
readFile("test4.txt", result4);
readFile("test5.txt", result5);
- assertTrue(expected1.equals(result1));
- assertTrue(expected2.equals(result2));
- assertTrue(expected3.equals(result3));
- assertTrue(expected4.equals(result4));
- assertTrue(expected5.equals(result5));
+ assertEquals(expected1,result1);
+ assertEquals(expected2,result2);
+ assertEquals(expected3,result3);
+ assertEquals(expected4,result4);
+ assertEquals(expected5,result5);
new File("test1.txt").delete();
new File("test2.txt").delete();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
index a18c447..054becc 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
@@ -60,7 +60,7 @@ public class FilterTest implements Serializable {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
+
env.fromElements(1, 2, 3, 4, 5, 6, 7).filter(new MyFilter()).addSink(new SetSink());
env.execute();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
new file mode 100644
index 0000000..dceaf46
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
@@ -0,0 +1,91 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.util.serialization;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.java.functions.MapFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.junit.Test;
+
+public class TypeSerializationTest {
+
+ private static class MyMap extends MapFunction<Tuple1<Integer>, Tuple1<String>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public Tuple1<String> map(Tuple1<Integer> value) throws Exception {
+ return null;
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void functionTypeSerializationTest() {
+ TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>> ser = new FunctionTypeWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>>(
+ new MyMap(), MapFunction.class, 0, -1, 1);
+
+ byte[] serializedType = SerializationUtils.serialize(ser);
+
+ TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>> ser2 = (TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>>) SerializationUtils
+ .deserialize(serializedType);
+
+ assertNotNull(ser.getInputTupleTypeInfo1());
+ assertNotNull(ser2.getInputTupleTypeInfo1());
+
+ assertNotNull(ser.getOutputTupleTypeInfo());
+ assertNotNull(ser2.getOutputTupleTypeInfo());
+
+ assertEquals(ser.getInputTupleTypeInfo1(), ser2.getInputTupleTypeInfo1());
+ assertEquals(ser.getInputTupleTypeInfo2(), ser2.getInputTupleTypeInfo2());
+ assertEquals(ser.getOutputTupleTypeInfo(), ser2.getOutputTupleTypeInfo());
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void objectTypeSerializationTest() {
+ Integer instance1 = new Integer(22);
+ Integer instance2 = null;
+ Integer instance3 = new Integer(34);
+
+ TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>> ser = new ObjectTypeWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>>(
+ instance1, instance2, instance3);
+
+ System.out.println(ser.getInputTupleTypeInfo1());
+
+ byte[] serializedType = SerializationUtils.serialize(ser);
+
+ TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>> ser2 = (TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>>) SerializationUtils
+ .deserialize(serializedType);
+
+ assertNotNull(ser.getInputTupleTypeInfo1());
+ assertNotNull(ser2.getInputTupleTypeInfo1());
+
+ assertNotNull(ser.getOutputTupleTypeInfo());
+ assertNotNull(ser2.getOutputTupleTypeInfo());
+
+ assertEquals(ser.getInputTupleTypeInfo1(), ser2.getInputTupleTypeInfo1());
+ assertEquals(ser.getInputTupleTypeInfo2(), ser2.getInputTupleTypeInfo2());
+ assertEquals(ser.getOutputTupleTypeInfo(), ser2.getOutputTupleTypeInfo());
+ }
+}
[39/51] [abbrv] [streaming] API update with more differentiated
DataStream types and javadoc + several fixes
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
deleted file mode 100644
index 76adf62..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ /dev/null
@@ -1,410 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-import org.apache.commons.lang3.SerializationException;
-import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.function.source.FileSourceFunction;
-import org.apache.flink.streaming.api.function.source.FileStreamFunction;
-import org.apache.flink.streaming.api.function.source.FromElementsFunction;
-import org.apache.flink.streaming.api.function.source.GenSequenceFunction;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.streaming.api.invokable.SourceInvokable;
-import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
-import org.apache.flink.streaming.util.serialization.ObjectTypeWrapper;
-
-/**
- * {@link ExecutionEnvironment} for streaming jobs. An instance of it is
- * necessary to construct streaming topologies.
- *
- */
-public abstract class StreamExecutionEnvironment {
-
- /**
- * The environment of the context (local by default, cluster if invoked
- * through command line)
- */
- private static StreamExecutionEnvironment contextEnvironment;
-
- /** flag to disable local executor when using the ContextEnvironment */
- private static boolean allowLocalExecution = true;
-
- private static int defaultLocalDop = Runtime.getRuntime().availableProcessors();
-
- private int degreeOfParallelism = 1;
-
- private int executionParallelism = -1;
-
- protected JobGraphBuilder jobGraphBuilder;
-
- // --------------------------------------------------------------------------------------------
- // Constructor and Properties
- // --------------------------------------------------------------------------------------------
-
- /**
- * Constructor for creating StreamExecutionEnvironment
- */
- protected StreamExecutionEnvironment() {
- jobGraphBuilder = new JobGraphBuilder("jobGraph");
- }
-
- public int getExecutionParallelism() {
- return executionParallelism == -1 ? degreeOfParallelism : executionParallelism;
- }
-
- /**
- * Gets the degree of parallelism with which operation are executed by
- * default. Operations can individually override this value to use a
- * specific degree of parallelism via {@link DataStream#setParallelism}.
- *
- * @return The degree of parallelism used by operations, unless they
- * override that value.
- */
- public int getDegreeOfParallelism() {
- return this.degreeOfParallelism;
- }
-
- /**
- * Sets the degree of parallelism (DOP) for operations executed through this
- * environment. Setting a DOP of x here will cause all operators (such as
- * map, batchReduce) to run with x parallel instances. This method overrides
- * the default parallelism for this environment. The
- * {@link LocalStreamEnvironment} uses by default a value equal to the
- * number of hardware contexts (CPU cores / threads). When executing the
- * program via the command line client from a JAR file, the default degree
- * of parallelism is the one configured for that setup.
- *
- * @param degreeOfParallelism
- * The degree of parallelism
- */
- protected void setDegreeOfParallelism(int degreeOfParallelism) {
- if (degreeOfParallelism < 1) {
- throw new IllegalArgumentException("Degree of parallelism must be at least one.");
- }
- this.degreeOfParallelism = degreeOfParallelism;
- }
-
- /**
- * Sets the number of hardware contexts (CPU cores / threads) used when
- * executed in {@link LocalStreamEnvironment}.
- *
- * @param degreeOfParallelism
- * The degree of parallelism in local environment
- */
- public void setExecutionParallelism(int degreeOfParallelism) {
- if (degreeOfParallelism < 1) {
- throw new IllegalArgumentException("Degree of parallelism must be at least one.");
- }
-
- this.executionParallelism = degreeOfParallelism;
- }
-
- // --------------------------------------------------------------------------------------------
- // Data stream creations
- // --------------------------------------------------------------------------------------------
-
- /**
- * Creates a DataStream that represents the Strings produced by reading the
- * given file line wise. The file will be read with the system's default
- * character set.
- *
- * @param filePath
- * The path of the file, as a URI (e.g.,
- * "file:///some/local/file" or "hdfs://host:port/file/path").
- * @return The DataStream representing the text file.
- */
- public DataStream<String> readTextFile(String filePath) {
- return addSource(new FileSourceFunction(filePath), 1);
- }
-
- public DataStream<String> readTextFile(String filePath, int parallelism) {
- return addSource(new FileSourceFunction(filePath), parallelism);
- }
-
- /**
- * Creates a DataStream that represents the Strings produced by reading the
- * given file line wise multiple times(infinite). The file will be read with
- * the system's default character set.
- *
- * @param filePath
- * The path of the file, as a URI (e.g.,
- * "file:///some/local/file" or "hdfs://host:port/file/path").
- * @return The DataStream representing the text file.
- */
- public DataStream<String> readTextStream(String filePath) {
- return addSource(new FileStreamFunction(filePath), 1);
- }
-
- public DataStream<String> readTextStream(String filePath, int parallelism) {
- return addSource(new FileStreamFunction(filePath), parallelism);
- }
-
- /**
- * Creates a new DataStream that contains the given elements. The elements
- * must all be of the same type, for example, all of the String or Integer.
- * The sequence of elements must not be empty. Furthermore, the elements
- * must be serializable (as defined in java.io.Serializable), because the
- * execution environment may ship the elements into the cluster.
- *
- * @param data
- * The collection of elements to create the DataStream from.
- * @param <OUT>
- * type of the returned stream
- * @return The DataStream representing the elements.
- */
- public <OUT extends Serializable> DataStream<OUT> fromElements(OUT... data) {
- DataStream<OUT> returnStream = new DataStream<OUT>(this, "elements");
-
- try {
- SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
- jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
- new ObjectTypeWrapper<OUT, Tuple, OUT>(data[0], null, data[0]), "source",
- SerializationUtils.serialize(function), 1);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize elements");
- }
- return returnStream;
- }
-
- /**
- * Creates a DataStream from the given non-empty collection. The type of the
- * DataStream is that of the elements in the collection. The elements need
- * to be serializable (as defined by java.io.Serializable), because the
- * framework may move the elements into the cluster if needed.
- *
- * @param data
- * The collection of elements to create the DataStream from.
- * @param <OUT>
- * type of the returned stream
- * @return The DataStream representing the elements.
- */
- @SuppressWarnings("unchecked")
- public <OUT extends Serializable> DataStream<OUT> fromCollection(Collection<OUT> data) {
- DataStream<OUT> returnStream = new DataStream<OUT>(this, "elements");
-
- if (data.isEmpty()) {
- throw new RuntimeException("Collection must not be empty");
- }
-
- try {
- SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
-
- jobGraphBuilder.addSource(
- returnStream.getId(),
- new SourceInvokable<OUT>(new FromElementsFunction<OUT>(data)),
- new ObjectTypeWrapper<OUT, Tuple, OUT>((OUT) data.toArray()[0], null, (OUT) data
- .toArray()[0]), "source", SerializationUtils.serialize(function), 1);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize collection");
- }
-
- return returnStream;
- }
-
- /**
- * Creates a new DataStream that contains a sequence of numbers.
- *
- * @param from
- * The number to start at (inclusive).
- * @param to
- * The number to stop at (inclusive)
- * @return A DataStrean, containing all number in the [from, to] interval.
- */
- public DataStream<Long> generateSequence(long from, long to) {
- return addSource(new GenSequenceFunction(from, to), 1);
- }
-
- /**
- * Ads a data source thus opening a {@link DataStream}.
- *
- * @param function
- * the user defined function
- * @param parallelism
- * number of parallel instances of the function
- * @param <OUT>
- * type of the returned stream
- * @return the data stream constructed
- */
- public <OUT> DataStream<OUT> addSource(SourceFunction<OUT> function, int parallelism) {
- DataStream<OUT> returnStream = new DataStream<OUT>(this, "source");
-
- try {
- jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
- new FunctionTypeWrapper<OUT, Tuple, OUT>(function, SourceFunction.class, 0, -1, 0),
- "source", SerializationUtils.serialize(function), parallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize SourceFunction");
- }
-
- return returnStream;
- }
-
- public <OUT> DataStream<OUT> addSource(SourceFunction<OUT> sourceFunction) {
- return addSource(sourceFunction, 1);
- }
-
- // --------------------------------------------------------------------------------------------
- // Instantiation of Execution Contexts
- // --------------------------------------------------------------------------------------------
-
- /**
- * Creates an execution environment that represents the context in which the
- * program is currently executed. If the program is invoked standalone, this
- * method returns a local execution environment, as returned by
- * {@link #createLocalEnvironment()}.
- *
- * @return The execution environment of the context in which the program is
- * executed.
- */
- public static StreamExecutionEnvironment getExecutionEnvironment() {
- return contextEnvironment == null ? createLocalEnvironment() : contextEnvironment;
- }
-
- /**
- * Creates a {@link LocalStreamEnvironment}. The local execution environment
- * will run the program in a multi-threaded fashion in the same JVM as the
- * environment was created in. The default degree of parallelism of the
- * local environment is the number of hardware contexts (CPU cores /
- * threads), unless it was specified differently by
- * {@link #setDegreeOfParallelism(int)}.
- *
- * @return A local execution environment.
- */
- public static LocalStreamEnvironment createLocalEnvironment() {
- return createLocalEnvironment(defaultLocalDop);
- }
-
- /**
- * Creates a {@link LocalStreamEnvironment}. The local execution environment
- * will run the program in a multi-threaded fashion in the same JVM as the
- * environment was created in. It will use the degree of parallelism
- * specified in the parameter.
- *
- * @param degreeOfParallelism
- * The degree of parallelism for the local environment.
- * @return A local execution environment with the specified degree of
- * parallelism.
- */
- public static LocalStreamEnvironment createLocalEnvironment(int degreeOfParallelism) {
- LocalStreamEnvironment lee = new LocalStreamEnvironment();
- lee.setDegreeOfParallelism(degreeOfParallelism);
- return lee;
- }
-
- // TODO:fix cluster default parallelism
- /**
- * Creates a {@link RemoteStreamEnvironment}. The remote environment sends
- * (parts of) the program to a cluster for execution. Note that all file
- * paths used in the program must be accessible from the cluster. The
- * execution will use no parallelism, unless the parallelism is set
- * explicitly via {@link #setDegreeOfParallelism}.
- *
- * @param host
- * The host name or address of the master (JobManager), where the
- * program should be executed.
- * @param port
- * The port of the master (JobManager), where the program should
- * be executed.
- * @param jarFiles
- * The JAR files with code that needs to be shipped to the
- * cluster. If the program uses user-defined functions,
- * user-defined input formats, or any libraries, those must be
- * provided in the JAR files.
- * @return A remote environment that executes the program on a cluster.
- */
- public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port,
- String... jarFiles) {
- return new RemoteStreamEnvironment(host, port, jarFiles);
- }
-
- /**
- * Creates a {@link RemoteStreamEnvironment}. The remote environment sends
- * (parts of) the program to a cluster for execution. Note that all file
- * paths used in the program must be accessible from the cluster. The
- * execution will use the specified degree of parallelism.
- *
- * @param host
- * The host name or address of the master (JobManager), where the
- * program should be executed.
- * @param port
- * The port of the master (JobManager), where the program should
- * be executed.
- * @param degreeOfParallelism
- * The degree of parallelism to use during the execution.
- * @param jarFiles
- * The JAR files with code that needs to be shipped to the
- * cluster. If the program uses user-defined functions,
- * user-defined input formats, or any libraries, those must be
- * provided in the JAR files.
- * @return A remote environment that executes the program on a cluster.
- */
- public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port,
- int degreeOfParallelism, String... jarFiles) {
- RemoteStreamEnvironment rec = new RemoteStreamEnvironment(host, port, jarFiles);
- rec.setDegreeOfParallelism(degreeOfParallelism);
- return rec;
- }
-
- // --------------------------------------------------------------------------------------------
- // Methods to control the context and local environments for execution from
- // packaged programs
- // --------------------------------------------------------------------------------------------
-
- protected static void initializeContextEnvironment(StreamExecutionEnvironment ctx) {
- contextEnvironment = ctx;
- }
-
- protected static boolean isContextEnvironmentSet() {
- return contextEnvironment != null;
- }
-
- protected static void disableLocalExecution() {
- allowLocalExecution = false;
- }
-
- public static boolean localExecutionIsAllowed() {
- return allowLocalExecution;
- }
-
- /**
- * Triggers the program execution. The environment will execute all parts of
- * the program that have resulted in a "sink" operation. Sink operations are
- * for example printing results or forwarding them to a message queue.
- * <p>
- * The program execution will be logged and displayed with a generated
- * default name.
- **/
- public abstract void execute();
-
- /**
- * Getter of the {@link JobGraphBuilder} of the streaming job.
- *
- * @return jobgraph
- */
- public JobGraphBuilder getJobGraphBuilder() {
- return jobGraphBuilder;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
deleted file mode 100755
index a39823c..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-/**
- * The StreamOperator represents a {@link DataStream} transformed with some user
- * defined operator.
- *
- * @param <OUT>
- * Output Type of the operator.
- */
-public class StreamOperator<OUT> extends DataStream<OUT> {
-
- protected StreamOperator(StreamExecutionEnvironment environment, String operatorType) {
- super(environment, operatorType);
- }
-
- protected StreamOperator(DataStream<OUT> dataStream) {
- super(dataStream);
- }
-
- @Override
- protected DataStream<OUT> copy() {
- return new StreamOperator<OUT>(this);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TwoInputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TwoInputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TwoInputStreamOperator.java
deleted file mode 100755
index a459dbf..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TwoInputStreamOperator.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api;
-
-/**
- * The TwoInputStreamOperator represents a {@link StreamOperator} with two
- * inputs.
- *
- * @param <IN1>
- * Type of the first input.
- *
- * @param <IN2>
- * Type of the second input.
- * @param <OUT>
- * Output Type of the operator.
- */
-public class TwoInputStreamOperator<IN1, IN2, OUT> extends StreamOperator<OUT> {
-
- protected TwoInputStreamOperator(StreamExecutionEnvironment environment, String operatorType) {
- super(environment, operatorType);
- }
-
- protected TwoInputStreamOperator(DataStream<OUT> dataStream) {
- super(dataStream);
- }
-
- @Override
- protected DataStream<OUT> copy() {
- return new TwoInputStreamOperator<IN1, IN2, OUT>(this);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
index 73a5749..ced3de7 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
@@ -19,7 +19,9 @@
package org.apache.flink.streaming.api.collector;
+import java.util.ArrayList;
import java.util.Collection;
+import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -39,6 +41,7 @@ public class DirectedStreamCollector<T> extends StreamCollector<T> {
OutputSelector<T> outputSelector;
private static final Log log = LogFactory.getLog(DirectedStreamCollector.class);
+ private List<RecordWriter<SerializationDelegate<StreamRecord<T>>>> emitted;
/**
* Creates a new DirectedStreamCollector
@@ -55,6 +58,7 @@ public class DirectedStreamCollector<T> extends StreamCollector<T> {
OutputSelector<T> outputSelector) {
super(channelID, serializationDelegate);
this.outputSelector = outputSelector;
+ this.emitted = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<T>>>>();
}
@@ -82,11 +86,14 @@ public class DirectedStreamCollector<T> extends StreamCollector<T> {
Collection<String> outputNames = outputSelector.getOutputs(streamRecord.getObject());
streamRecord.setId(channelID);
serializationDelegate.setInstance(streamRecord);
+ emitted.clear();
for (String outputName : outputNames) {
try {
- for (RecordWriter<SerializationDelegate<StreamRecord<T>>> output : outputMap
- .get(outputName)) {
+ RecordWriter<SerializationDelegate<StreamRecord<T>>> output = outputMap
+ .get(outputName);
+ if (!emitted.contains(output)) {
output.emit(serializationDelegate);
+ emitted.add(output);
}
} catch (Exception e) {
if (log.isErrorEnabled()) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
index 798d8fa..17d7e7b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
@@ -23,13 +23,16 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.datastream.SplitDataStream;
+
/**
- * Class for defining an OutputSelector for the directTo operator. Every output
- * object of a directed DataStream will run through this operator to select
- * outputs.
+ * Class for defining an OutputSelector for a {@link SplitDataStream} using the
+ * {@link SingleOutputStreamOperator#split} call. Every output object of a
+ * {@link SplitDataStream} will run through this operator to select outputs.
*
* @param <T>
- * Type parameter of the directed tuples/objects.
+ * Type parameter of the split values.
*/
public abstract class OutputSelector<T> implements Serializable {
private static final long serialVersionUID = 1L;
@@ -48,8 +51,9 @@ public abstract class OutputSelector<T> implements Serializable {
/**
* Method for selecting output names for the emitted objects when using the
- * directTo operator. The tuple will be emitted only to output names which
- * are added to the outputs collection.
+ * {@link SingleOutputStreamOperator#split} method. The values will be
+ * emitted only to output names which are added to the outputs collection.
+ * The outputs collection is cleared automatically after each select call.
*
* @param value
* Output object for which the output selection should be made.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
index 4317f75..20c3b78 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
@@ -34,7 +34,7 @@ import org.apache.flink.util.StringUtils;
/**
* Collector for tuples in Apache Flink stream processing. The collected
- * tuples/obecjts will be wrapped with ID in a {@link StreamRecord} and then
+ * values will be wrapped with ID in a {@link StreamRecord} and then
* emitted to the outputs.
*
* @param <T>
@@ -47,7 +47,7 @@ public class StreamCollector<T> implements Collector<T> {
protected StreamRecord<T> streamRecord;
protected int channelID;
private List<RecordWriter<SerializationDelegate<StreamRecord<T>>>> outputs;
- protected Map<String, List<RecordWriter<SerializationDelegate<StreamRecord<T>>>>> outputMap;
+ protected Map<String, RecordWriter<SerializationDelegate<StreamRecord<T>>>> outputMap;
protected SerializationDelegate<StreamRecord<T>> serializationDelegate;
/**
@@ -65,7 +65,7 @@ public class StreamCollector<T> implements Collector<T> {
this.streamRecord = new StreamRecord<T>();
this.channelID = channelID;
this.outputs = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<T>>>>();
- this.outputMap = new HashMap<String, List<RecordWriter<SerializationDelegate<StreamRecord<T>>>>>();
+ this.outputMap = new HashMap<String, RecordWriter<SerializationDelegate<StreamRecord<T>>>>();
}
/**
@@ -73,21 +73,19 @@ public class StreamCollector<T> implements Collector<T> {
*
* @param output
* The RecordWriter object representing the output.
- * @param outputName
- * User defined name of the output.
+ * @param outputNames
+ * User defined names of the output.
*/
public void addOutput(RecordWriter<SerializationDelegate<StreamRecord<T>>> output,
- String outputName) {
+ List<String> outputNames) {
outputs.add(output);
- if (outputName != null) {
- if (outputMap.containsKey(outputName)) {
- outputMap.get(outputName).add(output);
- } else {
- outputMap.put(outputName,
- new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<T>>>>());
- outputMap.get(outputName).add(output);
- }
+ for (String outputName : outputNames) {
+ if (outputName != null) {
+ if (!outputMap.containsKey(outputName)) {
+ outputMap.put(outputName, output);
+ }
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
new file mode 100755
index 0000000..c6cb8af
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
@@ -0,0 +1,132 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import java.io.Serializable;
+
+import org.apache.commons.lang3.SerializationException;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.streaming.api.JobGraphBuilder;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.function.co.CoMapFunction;
+import org.apache.flink.streaming.api.function.co.RichCoMapFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
+import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
+import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+
+/**
+ * The CoDataStream represents a stream for two different data types. It can be
+ * used to apply transformations like {@link CoMapFunction} on two
+ * {@link DataStream}s
+ *
+ * @param <IN1>
+ * Type of the first DataSteam.
+ * @param <IN2>
+ * Type of the second DataStream.
+ */
+public class CoDataStream<IN1, IN2> {
+
+ StreamExecutionEnvironment environment;
+ JobGraphBuilder jobGraphBuilder;
+ DataStream<IN1> input1;
+ DataStream<IN2> input2;
+
+ protected CoDataStream(StreamExecutionEnvironment environment, JobGraphBuilder jobGraphBuilder,
+ DataStream<IN1> input1, DataStream<IN2> input2) {
+ this.jobGraphBuilder = jobGraphBuilder;
+ this.environment = environment;
+ this.input1 = input1.copy();
+ this.input2 = input2.copy();
+ }
+
+ /**
+ * Returns the first {@link DataStream}.
+ *
+ * @return The first DataStream.
+ */
+ public DataStream<IN1> getFirst() {
+ return input1.copy();
+ }
+
+ /**
+ * Returns the second {@link DataStream}.
+ *
+ * @return The second DataStream.
+ */
+ public DataStream<IN2> getSecond() {
+ return input2.copy();
+ }
+
+ /**
+ * Applies a CoMap transformation on two separate {@link DataStream}s. The
+ * transformation calls a {@link CoMapFunction#map1} for each element
+ * of the first input and {@link CoMapFunction#map2} for each element
+ * of the second input. Each CoMapFunction call returns exactly one element.
+ * The user can also extend {@link RichCoMapFunction} to gain access to
+ * other features provided by the {@link RichFuntion} interface.
+ *
+ * @param coMapper
+ * The CoMapFunction used to jointly transform the two input
+ * DataStreams
+ * @return The transformed DataStream
+ */
+ public <OUT> SingleOutputStreamOperator<OUT, ?> map(CoMapFunction<IN1, IN2, OUT> coMapper) {
+ return addCoFunction("coMap", coMapper, new FunctionTypeWrapper<IN1, IN2, OUT>(coMapper,
+ CoMapFunction.class, 0, 1, 2), new CoMapInvokable<IN1, IN2, OUT>(coMapper));
+ }
+
+ protected <OUT> SingleOutputStreamOperator<OUT, ?> addCoFunction(String functionName,
+ final Function function, TypeSerializerWrapper<IN1, IN2, OUT> typeWrapper,
+ CoInvokable<IN1, IN2, OUT> functionInvokable) {
+
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(
+ environment, functionName);
+
+ try {
+ input1.jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, typeWrapper,
+ functionName, SerializationUtils.serialize((Serializable) function),
+ environment.getDegreeOfParallelism());
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize user defined function");
+ }
+
+ input1.connectGraph(input1, returnStream.getId(), 1);
+ input1.connectGraph(input2, returnStream.getId(), 2);
+
+ if ((input1.userDefinedName != null) && (input2.userDefinedName != null)) {
+ throw new RuntimeException("An operator cannot have two names");
+ } else {
+ if (input1.userDefinedName != null) {
+ returnStream.name(input1.getUserDefinedNames());
+ }
+
+ if (input2.userDefinedName != null) {
+ returnStream.name(input2.getUserDefinedNames());
+ }
+ }
+ // TODO consider iteration
+
+ return returnStream;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
new file mode 100755
index 0000000..d17990c
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
@@ -0,0 +1,98 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.partitioner.StreamPartitioner;
+
+/**
+ * The ConnectedDataStream represents a DataStream which consists of connected
+ * outputs of DataStreams of the same type. Operators applied on this will
+ * transform all the connected outputs jointly.
+ *
+ * @param <OUT>
+ * Type of the output.
+ */
+public class ConnectedDataStream<OUT> extends DataStream<OUT> {
+
+ protected List<DataStream<OUT>> connectedStreams;
+
+ protected ConnectedDataStream(StreamExecutionEnvironment environment, String operatorType) {
+ super(environment, operatorType);
+ this.connectedStreams = new ArrayList<DataStream<OUT>>();
+ this.connectedStreams.add(this);
+ }
+
+ protected ConnectedDataStream(DataStream<OUT> dataStream) {
+ super(dataStream);
+ connectedStreams = new ArrayList<DataStream<OUT>>();
+ if (dataStream instanceof ConnectedDataStream) {
+ for (DataStream<OUT> stream : ((ConnectedDataStream<OUT>) dataStream).connectedStreams) {
+ connectedStreams.add(stream);
+ }
+ } else {
+ this.connectedStreams.add(this);
+ }
+
+ }
+
+ // @Override
+ // public IterativeDataStream<OUT> iterate() {
+ // throw new RuntimeException("Cannot iterate connected DataStreams");
+ // }
+
+ protected void addConnection(DataStream<OUT> stream) {
+ if ((stream.userDefinedName != null) || (this.userDefinedName != null)) {
+ if (!this.userDefinedName.equals(stream.userDefinedName)) {
+ throw new RuntimeException("Error: Connected NamedDataStreams must have same names");
+ }
+ }
+ connectedStreams.add(stream.copy());
+ }
+
+ @Override
+ protected List<String> getUserDefinedNames() {
+ List<String> nameList = new ArrayList<String>();
+ for (DataStream<OUT> stream : connectedStreams) {
+ nameList.add(stream.userDefinedName);
+ }
+ return nameList;
+ }
+
+ @Override
+ protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
+ ConnectedDataStream<OUT> returnStream = (ConnectedDataStream<OUT>) this.copy();
+
+ for (DataStream<OUT> stream : returnStream.connectedStreams) {
+ stream.partitioner = partitioner;
+ }
+
+ return returnStream;
+ }
+
+ @Override
+ protected ConnectedDataStream<OUT> copy() {
+ return new ConnectedDataStream<OUT>(this);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
new file mode 100644
index 0000000..b692984
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -0,0 +1,852 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.SerializationException;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.functions.RichFilterFunction;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.JobGraphBuilder;
+import org.apache.flink.streaming.api.collector.OutputSelector;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.function.sink.PrintSinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
+import org.apache.flink.streaming.api.function.sink.WriteFormatAsText;
+import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByBatches;
+import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByMillis;
+import org.apache.flink.streaming.api.invokable.SinkInvokable;
+import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
+import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
+import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
+import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
+import org.apache.flink.streaming.api.invokable.operator.WindowReduceInvokable;
+import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
+import org.apache.flink.streaming.partitioner.DistributePartitioner;
+import org.apache.flink.streaming.partitioner.FieldsPartitioner;
+import org.apache.flink.streaming.partitioner.ForwardPartitioner;
+import org.apache.flink.streaming.partitioner.ShufflePartitioner;
+import org.apache.flink.streaming.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
+import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+
+/**
+ * A DataStream represents a stream of elements of the same type. A DataStream
+ * can be transformed into another DataStream by applying a transformation as
+ * for example
+ * <ul>
+ * <li>{@link DataStream#map},</li>
+ * <li>{@link DataStream#filter}, or</li>
+ * <li>{@link DataStream#batchReduce}.</li>
+ * </ul>
+ *
+ * @param <OUT>
+ * The type of the DataStream, i.e., the type of the elements of the
+ * DataStream.
+ */
+public abstract class DataStream<OUT> {
+
+ protected static Integer counter = 0;
+ protected final StreamExecutionEnvironment environment;
+ protected final String id;
+ protected int degreeOfParallelism;
+ protected String userDefinedName;
+ protected StreamPartitioner<OUT> partitioner;
+
+ protected final JobGraphBuilder jobGraphBuilder;
+
+ /**
+ * Create a new {@link DataStream} in the given execution environment with
+ * partitioning set to forward by default.
+ *
+ * @param environment
+ * StreamExecutionEnvironment
+ * @param operatorType
+ * The type of the operator in the component
+ */
+ public DataStream(StreamExecutionEnvironment environment, String operatorType) {
+ if (environment == null) {
+ throw new NullPointerException("context is null");
+ }
+
+ // TODO add name based on component number an preferable sequential id
+ counter++;
+ this.id = operatorType + "-" + counter.toString();
+ this.environment = environment;
+ this.degreeOfParallelism = environment.getDegreeOfParallelism();
+ this.jobGraphBuilder = environment.getJobGraphBuilder();
+ this.partitioner = new ForwardPartitioner<OUT>();
+
+ }
+
+ /**
+ * Create a new DataStream by creating a copy of another DataStream
+ *
+ * @param dataStream
+ * The DataStream that will be copied.
+ */
+ public DataStream(DataStream<OUT> dataStream) {
+ this.environment = dataStream.environment;
+ this.id = dataStream.id;
+ this.degreeOfParallelism = dataStream.degreeOfParallelism;
+ this.userDefinedName = dataStream.userDefinedName;
+ this.partitioner = dataStream.partitioner;
+ this.jobGraphBuilder = dataStream.jobGraphBuilder;
+
+ }
+
+ /**
+ * Partitioning strategy on the stream.
+ */
+ public static enum ConnectionType {
+ SHUFFLE, BROADCAST, FIELD, FORWARD, DISTRIBUTE
+ }
+
+ /**
+ * Returns the ID of the {@link DataStream}.
+ *
+ * @return ID of the DataStream
+ */
+ public String getId() {
+ return id;
+ }
+
+ /**
+ * Gets the degree of parallelism for this operator.
+ *
+ * @return The parallelism set for this operator.
+ */
+ public int getParallelism() {
+ return this.degreeOfParallelism;
+ }
+
+ /**
+ * Creates a new by connecting {@link DataStream} outputs of the same type
+ * with each other. The DataStreams connected using this operator will be
+ * transformed simultaneously.
+ *
+ * @param streams
+ * The DataStreams to connect output with.
+ * @return The {@link ConnectedDataStream}.
+ */
+ public ConnectedDataStream<OUT> connectWith(DataStream<OUT>... streams) {
+ ConnectedDataStream<OUT> returnStream = new ConnectedDataStream<OUT>(this);
+
+ for (DataStream<OUT> stream : streams) {
+ returnStream.addConnection(stream);
+ }
+ return returnStream;
+ }
+
+ /**
+ * Creates a new {@link CoDataStream} bye connecting {@link DataStream}
+ * outputs of different type with each other. The DataStreams connected
+ * using this operators can be used with CoFunctions.
+ *
+ * @param dataStream
+ * The DataStream with which this stream will be joined.
+ * @return The {@link CoDataStream}.
+ */
+ public <R> CoDataStream<OUT, R> co(DataStream<R> dataStream) {
+ return new CoDataStream<OUT, R>(environment, jobGraphBuilder, this, dataStream);
+ }
+
+ /**
+ * Sets the partitioning of the {@link DataStream} so that the output tuples
+ * are partitioned by their hashcode and are sent to only one component.
+ *
+ * @param keyposition
+ * The field used to compute the hashcode.
+ * @return The DataStream with field partitioning set.
+ */
+ public DataStream<OUT> partitionBy(int keyposition) {
+ if (keyposition < 0) {
+ throw new IllegalArgumentException("The position of the field must be non-negative");
+ }
+
+ return setConnectionType(new FieldsPartitioner<OUT>(keyposition));
+ }
+
+ /**
+ * Sets the partitioning of the {@link DataStream} so that the output tuples
+ * are broadcasted to every parallel instance of the next component.
+ *
+ * @return The DataStream with broadcast partitioning set.
+ */
+ public DataStream<OUT> broadcast() {
+ return setConnectionType(new BroadcastPartitioner<OUT>());
+ }
+
+ /**
+ * Sets the partitioning of the {@link DataStream} so that the output tuples
+ * are shuffled to the next component.
+ *
+ * @return The DataStream with shuffle partitioning set.
+ */
+ public DataStream<OUT> shuffle() {
+ return setConnectionType(new ShufflePartitioner<OUT>());
+ }
+
+ /**
+ * Sets the partitioning of the {@link DataStream} so that the output tuples
+ * are forwarded to the local subtask of the next component. This is the
+ * default partitioner setting.
+ *
+ * @return The DataStream with shuffle partitioning set.
+ */
+ public DataStream<OUT> forward() {
+ return setConnectionType(new ForwardPartitioner<OUT>());
+ }
+
+ /**
+ * Sets the partitioning of the {@link DataStream} so that the output tuples
+ * are distributed evenly to the next component.
+ *
+ * @return The DataStream with shuffle partitioning set.
+ */
+ public DataStream<OUT> distribute() {
+ return setConnectionType(new DistributePartitioner<OUT>());
+ }
+
+ /**
+ * Applies a Map transformation on a {@link DataStream}. The transformation
+ * calls a {@link MapFunction} for each element of the DataStream. Each
+ * MapFunction call returns exactly one element. The user can also extend
+ * {@link RichMapFunction} to gain access to other features provided by the
+ * {@link RichFuntion} interface.
+ *
+ * @param mapper
+ * The MapFunction that is called for each element of the
+ * DataStream.
+ * @param <R>
+ * output type
+ * @return The transformed {@link DataStream}.
+ */
+ public <R> SingleOutputStreamOperator<R, ?> map(MapFunction<OUT, R> mapper) {
+ return addFunction("map", mapper, new FunctionTypeWrapper<OUT, Tuple, R>(mapper,
+ MapFunction.class, 0, -1, 1), new MapInvokable<OUT, R>(mapper));
+ }
+
+ /**
+ * Applies a FlatMap transformation on a {@link DataStream}. The
+ * transformation calls a {@link FlatMapFunction} for each element of the
+ * DataStream. Each FlatMapFunction call can return any number of elements
+ * including none. The user can also extend {@link RichFlatMapFunction} to
+ * gain access to other features provided by the {@link RichFuntion}
+ * interface.
+ *
+ * @param flatMapper
+ * The FlatMapFunction that is called for each element of the
+ * DataStream
+ *
+ * @param <R>
+ * output type
+ * @return The transformed {@link DataStream}.
+ */
+ public <R> SingleOutputStreamOperator<R, ?> flatMap(FlatMapFunction<OUT, R> flatMapper) {
+ return addFunction("flatMap", flatMapper, new FunctionTypeWrapper<OUT, Tuple, R>(
+ flatMapper, FlatMapFunction.class, 0, -1, 1), new FlatMapInvokable<OUT, R>(
+ flatMapper));
+ }
+
+ /**
+ * Applies a reduce transformation on preset chunks of the DataStream. The
+ * transformation calls a {@link GroupReduceFunction} for each tuple batch
+ * of the predefined size. Each GroupReduceFunction call can return any
+ * number of elements including none. The user can also extend
+ * {@link RichGroupReduceFunction} to gain access to other features provided
+ * by the {@link RichFuntion} interface.
+ *
+ *
+ * @param reducer
+ * The GroupReduceFunction that is called for each tuple batch.
+ * @param batchSize
+ * The number of tuples grouped together in the batch.
+ * @param <R>
+ * output type
+ * @return The transformed {@link DataStream}.
+ */
+ public <R> SingleOutputStreamOperator<R, ?> batchReduce(GroupReduceFunction<OUT, R> reducer,
+ int batchSize) {
+ return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
+ GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<OUT, R>(reducer,
+ batchSize));
+ }
+
+ /**
+ * Applies a reduce transformation on preset "time" chunks of the
+ * DataStream. The transformation calls a {@link GroupReduceFunction} on
+ * records received during the predefined time window. The window shifted
+ * after each reduce call. Each GroupReduceFunction call can return any
+ * number of elements including none.The user can also extend
+ * {@link RichGroupReduceFunction} to gain access to other features provided
+ * by the {@link RichFuntion} interface.
+ *
+ *
+ * @param reducer
+ * The GroupReduceFunction that is called for each time window.
+ * @param windowSize
+ * The time window to run the reducer on, in milliseconds.
+ * @param <R>
+ * output type
+ * @return The transformed DataStream.
+ */
+ public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
+ long windowSize) {
+ return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
+ GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<OUT, R>(reducer,
+ windowSize));
+ }
+
+ /**
+ * Applies a Filter transformation on a {@link DataStream}. The
+ * transformation calls a {@link FilterFunction} for each element of the
+ * DataStream and retains only those element for which the function returns
+ * true. Elements for which the function returns false are filtered. The
+ * user can also extend {@link RichFilterFunction} to gain access to other
+ * features provided by the {@link RichFuntion} interface.
+ *
+ * @param filter
+ * The FilterFunction that is called for each element of the
+ * DataSet.
+ * @return The filtered DataStream.
+ */
+ public SingleOutputStreamOperator<OUT, ?> filter(FilterFunction<OUT> filter) {
+ return addFunction("filter", filter, new FunctionTypeWrapper<OUT, Tuple, OUT>(filter,
+ FilterFunction.class, 0, -1, 0), new FilterInvokable<OUT>(filter));
+ }
+
+ /**
+ * Writes a DataStream to the standard output stream (stdout). For each
+ * element of the DataStream the result of {@link Object#toString()} is
+ * written.
+ *
+ * @return The closed DataStream.
+ */
+ public DataStream<OUT> print() {
+ DataStream<OUT> inputStream = this.copy();
+ PrintSinkFunction<OUT> printFunction = new PrintSinkFunction<OUT>();
+ DataStream<OUT> returnStream = addSink(inputStream, printFunction, null);
+
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+
+ return returnStream;
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsText(String path) {
+ return writeAsText(this, path, new WriteFormatAsText<OUT>(), 1, null);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically, in every millis milliseconds. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param millis
+ * is the file update frequency
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsText(String path, long millis) {
+ return writeAsText(this, path, new WriteFormatAsText<OUT>(), millis, null);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically in equally sized batches. For every
+ * element of the DataStream the result of {@link Object#toString()} is
+ * written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param batchSize
+ * is the size of the batches, i.e. the number of tuples written
+ * to the file at a time
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsText(String path, int batchSize) {
+ return writeAsText(this, path, new WriteFormatAsText<OUT>(), batchSize, null);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically, in every millis milliseconds. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param millis
+ * is the file update frequency
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * system time.
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsText(String path, long millis, OUT endTuple) {
+ return writeAsText(this, path, new WriteFormatAsText<OUT>(), millis, endTuple);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically in equally sized batches. For every
+ * element of the DataStream the result of {@link Object#toString()} is
+ * written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param batchSize
+ * is the size of the batches, i.e. the number of tuples written
+ * to the file at a time
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * batchSize.
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsText(String path, int batchSize, OUT endTuple) {
+ return writeAsText(this, path, new WriteFormatAsText<OUT>(), batchSize, endTuple);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically, in every millis milliseconds. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param millis
+ * is the file update frequency
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * system time.
+ *
+ * @return the data stream constructed
+ */
+ private DataStream<OUT> writeAsText(DataStream<OUT> inputStream, String path,
+ WriteFormatAsText<OUT> format, long millis, OUT endTuple) {
+ DataStream<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<OUT>(
+ path, format, millis, endTuple), null);
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
+ return returnStream;
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically in equally sized batches. For every
+ * element of the DataStream the result of {@link Object#toString()} is
+ * written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param batchSize
+ * is the size of the batches, i.e. the number of tuples written
+ * to the file at a time
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * batchSize.
+ *
+ * @return the data stream constructed
+ */
+ private DataStream<OUT> writeAsText(DataStream<OUT> inputStream, String path,
+ WriteFormatAsText<OUT> format, int batchSize, OUT endTuple) {
+ DataStream<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<OUT>(
+ path, format, batchSize, endTuple), null);
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
+ return returnStream;
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsCsv(String path) {
+ return writeAsCsv(this, path, new WriteFormatAsCsv<OUT>(), 1, null);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically, in every millis milliseconds. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param millis
+ * is the file update frequency
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsCsv(String path, long millis) {
+ return writeAsCsv(this, path, new WriteFormatAsCsv<OUT>(), millis, null);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically in equally sized batches. For every
+ * element of the DataStream the result of {@link Object#toString()} is
+ * written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param batchSize
+ * is the size of the batches, i.e. the number of tuples written
+ * to the file at a time
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsCsv(String path, int batchSize) {
+ return writeAsCsv(this, path, new WriteFormatAsCsv<OUT>(), batchSize, null);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically, in every millis milliseconds. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param millis
+ * is the file update frequency
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * system time.
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsCsv(String path, long millis, OUT endTuple) {
+ return writeAsCsv(this, path, new WriteFormatAsCsv<OUT>(), millis, endTuple);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically in equally sized batches. For every
+ * element of the DataStream the result of {@link Object#toString()} is
+ * written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param batchSize
+ * is the size of the batches, i.e. the number of tuples written
+ * to the file at a time
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * batchSize.
+ *
+ * @return The closed DataStream
+ */
+ public DataStream<OUT> writeAsCsv(String path, int batchSize, OUT endTuple) {
+ if (this instanceof SingleOutputStreamOperator) {
+ ((SingleOutputStreamOperator<?, ?>) this).setMutability(false);
+ }
+ return writeAsCsv(this, path, new WriteFormatAsCsv<OUT>(), batchSize, endTuple);
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in csv format. The
+ * writing is performed periodically, in every millis milliseconds. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param millis
+ * is the file update frequency
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * system time.
+ *
+ * @return the data stream constructed
+ */
+ private DataStream<OUT> writeAsCsv(DataStream<OUT> inputStream, String path,
+ WriteFormatAsCsv<OUT> format, long millis, OUT endTuple) {
+ DataStream<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<OUT>(
+ path, format, millis, endTuple));
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
+ return returnStream;
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in csv format. The
+ * writing is performed periodically in equally sized batches. For every
+ * element of the DataStream the result of {@link Object#toString()} is
+ * written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param batchSize
+ * is the size of the batches, i.e. the number of tuples written
+ * to the file at a time
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * batchSize.
+ *
+ * @return the data stream constructed
+ */
+ private DataStream<OUT> writeAsCsv(DataStream<OUT> inputStream, String path,
+ WriteFormatAsCsv<OUT> format, int batchSize, OUT endTuple) {
+ DataStream<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<OUT>(
+ path, format, batchSize, endTuple), null);
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
+ return returnStream;
+ }
+
+ /**
+ * Initiates an iterative part of the program that executes multiple times
+ * and feeds back data streams. The iterative part needs to be closed by
+ * calling {@link IterativeDataStream#closeWith(DataStream)}. The
+ * transformation of this IterativeDataStream will be the iteration head.
+ * The data stream given to the {@code closeWith(DataStream)} method is the
+ * data stream that will be fed back and used as the input for the iteration
+ * head. Unlike in batch processing by default the output of the iteration
+ * stream is directed to both to the iteration head and the next component.
+ * To direct tuples to the iteration head or the output specifically one can
+ * use the {@code split(OutputSelector)} on the iteration tail while
+ * referencing the iteration head as 'iterate'.
+ *
+ * The iteration edge will be partitioned the same way as the first input of
+ * the iteration head.
+ *
+ * @return The iterative data stream created.
+ */
+ public IterativeDataStream<OUT> iterate() {
+ return new IterativeDataStream<OUT>(this);
+ }
+
+ protected <R> DataStream<OUT> addIterationSource(String iterationID) {
+
+ DataStream<R> returnStream = new DataStreamSource<R>(environment, "iterationSource");
+
+ jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
+ degreeOfParallelism);
+
+ return this.copy();
+ }
+
+ /**
+ * Internal function for passing the user defined functions to the JobGraph
+ * of the job.
+ *
+ * @param functionName
+ * name of the function
+ * @param function
+ * the user defined function
+ * @param functionInvokable
+ * the wrapping JobVertex instance
+ * @param <R>
+ * type of the return stream
+ * @return the data stream constructed
+ */
+ private <R> SingleOutputStreamOperator<R, ?> addFunction(String functionName,
+ final Function function, TypeSerializerWrapper<OUT, Tuple, R> typeWrapper,
+ UserTaskInvokable<OUT, R> functionInvokable) {
+
+ DataStream<OUT> inputStream = this.copy();
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ SingleOutputStreamOperator<R, ?> returnStream = new SingleOutputStreamOperator(environment,
+ functionName);
+
+ try {
+ jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, typeWrapper,
+ functionName, SerializationUtils.serialize((Serializable) function),
+ degreeOfParallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize user defined function");
+ }
+
+ connectGraph(inputStream, returnStream.getId(), 0);
+
+ if (inputStream instanceof IterativeDataStream) {
+ returnStream.addIterationSource(((IterativeDataStream<OUT>) inputStream).iterationID
+ .toString());
+ }
+
+ if (userDefinedName != null) {
+ returnStream.name(getUserDefinedNames());
+ }
+
+ return returnStream;
+ }
+
+ protected List<String> getUserDefinedNames() {
+ List<String> nameList = new ArrayList<String>();
+ nameList.add(userDefinedName);
+ return nameList;
+ }
+
+ /**
+ * Gives the data transformation(vertex) a user defined name in order to use
+ * with directed outputs. The {@link OutputSelector} of the input vertex
+ * should use this name for directed emits.
+ *
+ * @param name
+ * The name to set
+ * @return The named DataStream.
+ */
+ protected DataStream<OUT> name(List<String> name) {
+
+ userDefinedName = name.get(0);
+ jobGraphBuilder.setUserDefinedName(id, name);
+
+ return this;
+ }
+
+ /**
+ * Internal function for setting the partitioner for the DataStream
+ *
+ * @param partitioner
+ * Partitioner to set.
+ * @return The modified DataStream.
+ */
+ protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
+ DataStream<OUT> returnStream = this.copy();
+
+ returnStream.partitioner = partitioner;
+
+ return returnStream;
+ }
+
+ /**
+ * Internal function for assembling the underlying
+ * {@link org.apache.flink.nephele.jobgraph.JobGraph} of the job. Connects
+ * the outputs of the given input stream to the specified output stream
+ * given by the outputID.
+ *
+ * @param inputStream
+ * input data stream
+ * @param outputID
+ * ID of the output
+ * @param typeNumber
+ * Number of the type (used at co-functions)
+ */
+ protected <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
+ if (inputStream instanceof ConnectedDataStream) {
+ for (DataStream<X> stream : ((ConnectedDataStream<X>) inputStream).connectedStreams) {
+ jobGraphBuilder.setEdge(stream.getId(), outputID, stream.partitioner, typeNumber);
+ }
+ } else {
+ jobGraphBuilder.setEdge(inputStream.getId(), outputID, inputStream.partitioner,
+ typeNumber);
+ }
+
+ }
+
+ /**
+ * Adds the given sink to this DataStream. Only streams with sinks added
+ * will be executed once the {@link StreamExecutionEnvironment#execute()}
+ * method is called.
+ *
+ * @param sinkFunction
+ * The object containing the sink's invoke function.
+ * @return The closed DataStream.
+ */
+ public DataStream<OUT> addSink(SinkFunction<OUT> sinkFunction) {
+ return addSink(this.copy(), sinkFunction);
+ }
+
+ private DataStream<OUT> addSink(DataStream<OUT> inputStream, SinkFunction<OUT> sinkFunction) {
+ return addSink(inputStream, sinkFunction, new FunctionTypeWrapper<OUT, Tuple, OUT>(
+ sinkFunction, SinkFunction.class, 0, -1, 0));
+ }
+
+ private DataStream<OUT> addSink(DataStream<OUT> inputStream, SinkFunction<OUT> sinkFunction,
+ TypeSerializerWrapper<OUT, Tuple, OUT> typeWrapper) {
+ DataStream<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink");
+
+ try {
+ jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<OUT>(sinkFunction),
+ typeWrapper, "sink", SerializationUtils.serialize(sinkFunction),
+ degreeOfParallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize SinkFunction");
+ }
+
+ inputStream.connectGraph(inputStream, returnStream.getId(), 0);
+
+ if (this.copy().userDefinedName != null) {
+ returnStream.name(getUserDefinedNames());
+ }
+
+ return returnStream;
+ }
+
+ /**
+ * Creates a copy of the {@link DataStream}
+ *
+ * @return The copy
+ */
+ protected abstract DataStream<OUT> copy();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
new file mode 100755
index 0000000..ee6502f
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
@@ -0,0 +1,45 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+/**
+ * Represents the end of a DataStream.
+ *
+ * @param <IN>
+ * The type of the DataStream closed by the sink.
+ */
+public class DataStreamSink<IN> extends DataStream<IN> {
+
+ protected DataStreamSink(StreamExecutionEnvironment environment, String operatorType) {
+ super(environment, operatorType);
+ }
+
+ protected DataStreamSink(DataStream<IN> dataStream) {
+ super(dataStream);
+ }
+
+ @Override
+ protected DataStream<IN> copy() {
+ throw new RuntimeException("Data stream sinks cannot be copied");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
new file mode 100755
index 0000000..f939851
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
@@ -0,0 +1,39 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+/**
+ * The DataStreamSource represents the starting point of a DataStream.
+ *
+ * @param <OUT>
+ * Type of the DataStream created.
+ */
+public class DataStreamSource<OUT> extends SingleOutputStreamOperator<OUT, DataStreamSource<OUT>> {
+
+ public DataStreamSource(StreamExecutionEnvironment environment, String operatorType) {
+ super(environment, operatorType);
+ }
+
+ public DataStreamSource(DataStream<OUT> dataStream) {
+ super(dataStream);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
new file mode 100644
index 0000000..b9aadcd
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
@@ -0,0 +1,103 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.streaming.partitioner.ForwardPartitioner;
+
+/**
+ * The iterative data stream represents the start of an iteration in a
+ * {@link DataStream}.
+ *
+ * @param <T>
+ * Type of the DataStream
+ */
+public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, IterativeDataStream<T>> {
+
+ static Integer iterationCount = 0;
+ protected Integer iterationID;
+
+ protected IterativeDataStream(DataStream<T> dataStream) {
+ super(dataStream);
+ iterationID = iterationCount;
+ iterationCount++;
+ }
+
+ protected IterativeDataStream(DataStream<T> dataStream, Integer iterationID) {
+ super(dataStream);
+ this.iterationID = iterationID;
+ }
+
+ /**
+ * Closes the iteration. This method defines the end of the iterative
+ * program part. By default the DataStream represented by the parameter will
+ * be fed back to the iteration head, however the user can explicitly select
+ * which tuples should be iterated by {@code directTo(OutputSelector)}.
+ * Tuples directed to 'iterate' will be fed back to the iteration head.
+ *
+ * @param iterationResult
+ * The data stream that can be fed back to the next iteration.
+ *
+ */
+ public DataStream<T> closeWith(DataStream<T> iterationResult) {
+ return closeWith(iterationResult, null);
+ }
+
+ /**
+ * Closes the iteration. This method defines the end of the iterative
+ * program part. By default the DataStream represented by the parameter will
+ * be fed back to the iteration head, however the user can explicitly select
+ * which tuples should be iterated by {@code directTo(OutputSelector)}.
+ * Tuples directed to 'iterate' will be fed back to the iteration head.
+ *
+ * @param iterationTail
+ * The data stream that can be fed back to the next iteration.
+ * @param iterationName
+ * Name of the iteration edge (backward edge to iteration head)
+ * when used with directed emits
+ *
+ */
+ public <R> DataStream<T> closeWith(DataStream<T> iterationTail, String iterationName) {
+ DataStream<R> returnStream = new DataStreamSink<R>(environment, "iterationSink");
+
+ jobGraphBuilder.addIterationSink(returnStream.getId(), iterationTail.getId(),
+ iterationID.toString(), iterationTail.getParallelism(), iterationName);
+
+ jobGraphBuilder.setIterationSourceParallelism(iterationID.toString(),
+ iterationTail.getParallelism());
+
+ if (iterationTail instanceof ConnectedDataStream) {
+ for (DataStream<T> stream : ((ConnectedDataStream<T>) iterationTail).connectedStreams) {
+ String inputID = stream.getId();
+ jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<T>(),
+ 0);
+ }
+ } else {
+ jobGraphBuilder.setEdge(iterationTail.getId(), returnStream.getId(),
+ new ForwardPartitioner<T>(), 0);
+ }
+
+ return iterationTail;
+ }
+
+ @Override
+ protected IterativeDataStream<T> copy() {
+ return new IterativeDataStream<T>(this, iterationID);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
new file mode 100755
index 0000000..9af4dc8
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -0,0 +1,148 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.commons.lang3.SerializationException;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.streaming.api.collector.OutputSelector;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+/**
+ * The SingleOutputStreamOperator represents a user defined transformation
+ * applied on a {@link DataStream} with one predefined output type.
+ *
+ * @param <OUT>
+ * Output type of the operator.
+ * @param <O>
+ * Type of the operator.
+ */
+public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperator<OUT, O>> extends
+ DataStream<OUT> {
+
+ protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, String operatorType) {
+ super(environment, operatorType);
+ }
+
+ protected SingleOutputStreamOperator(DataStream<OUT> dataStream) {
+ super(dataStream);
+ }
+
+ /**
+ * Sets the degree of parallelism for this operator. The degree must be 1 or
+ * more.
+ *
+ * @param dop
+ * The degree of parallelism for this operator.
+ * @return The operator with set degree of parallelism.
+ */
+ public SingleOutputStreamOperator<OUT, O> setParallelism(int dop) {
+ if (dop < 1) {
+ throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
+ }
+ this.degreeOfParallelism = dop;
+
+ jobGraphBuilder.setParallelism(id, degreeOfParallelism);
+
+ return this;
+ }
+
+ /**
+ * Sets the mutability of the operator. If the operator is set to mutable,
+ * the tuples received in the user defined functions, will be reused after
+ * the function call. Setting an operator to mutable reduces garbage
+ * collection overhead and thus increases scalability. Please note that if a
+ * {@link DataStream#batchReduce} or {@link DataStream#windowReduce} is used
+ * as mutable, the user can only iterate through the iterator once in every
+ * invoke.
+ *
+ * @param isMutable
+ * The mutability of the operator.
+ * @return The operator with mutability set.
+ */
+ public DataStream<OUT> setMutability(boolean isMutable) {
+ jobGraphBuilder.setMutability(id, isMutable);
+ return this;
+ }
+
+ /**
+ * Sets the maximum time frequency (ms) for the flushing of the output
+ * buffer. By default the output buffers flush only when they are full.
+ *
+ * @param timeoutMillis
+ * The maximum time between two output flushes.
+ * @return The operator with buffer timeout set.
+ */
+ public DataStream<OUT> setBufferTimeout(long timeoutMillis) {
+ jobGraphBuilder.setBufferTimeout(id, timeoutMillis);
+ return this;
+ }
+
+ /**
+ * Operator used for directing tuples to specific named outputs using an
+ * {@link OutputSelector}. Calling this method on an operator creates a new
+ * {@link SplitDataStream}.
+ *
+ * @param outputSelector
+ * The user defined {@link OutputSelector} for directing the
+ * tuples.
+ * @return The {@link SplitDataStream}
+ */
+ public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector) {
+ try {
+ jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
+
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize OutputSelector");
+ }
+
+ return new SplitDataStream<OUT>(this);
+ }
+
+ @SuppressWarnings("unchecked")
+ public SingleOutputStreamOperator<OUT, O> partitionBy(int keyposition) {
+ return (SingleOutputStreamOperator<OUT, O>) super.partitionBy(keyposition);
+ }
+
+ @SuppressWarnings("unchecked")
+ public SingleOutputStreamOperator<OUT, O> broadcast() {
+ return (SingleOutputStreamOperator<OUT, O>) super.broadcast();
+ }
+
+ @SuppressWarnings("unchecked")
+ public SingleOutputStreamOperator<OUT, O> shuffle() {
+ return (SingleOutputStreamOperator<OUT, O>) super.shuffle();
+ }
+
+ @SuppressWarnings("unchecked")
+ public SingleOutputStreamOperator<OUT, O> forward() {
+ return (SingleOutputStreamOperator<OUT, O>) super.forward();
+ }
+
+ @SuppressWarnings("unchecked")
+ public SingleOutputStreamOperator<OUT, O> distribute() {
+ return (SingleOutputStreamOperator<OUT, O>) super.distribute();
+ }
+
+ @Override
+ protected SingleOutputStreamOperator<OUT, O> copy() {
+ return new SingleOutputStreamOperator<OUT, O>(this);
+ }
+
+}
[35/51] [abbrv] [streaming] Updated API to use RichFunctions
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
index f3e9310..6295d02 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
@@ -19,10 +19,10 @@
package org.apache.flink.streaming.examples.wordcount;
-import org.apache.flink.api.java.functions.FlatMapFunction;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.util.Collector;
-public class WordCountSplitter extends FlatMapFunction<String, String> {
+public class WordCountSplitter extends RichFlatMapFunction<String, String> {
private static final long serialVersionUID = 1L;
// Splits the lines according on spaces
[36/51] [abbrv] git commit: [streaming] Updated API to use
RichFunctions
Posted by se...@apache.org.
[streaming] Updated API to use RichFunctions
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/776bd3f6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/776bd3f6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/776bd3f6
Branch: refs/heads/master
Commit: 776bd3f6dbfd85db978ff4d5785935a19f43d77b
Parents: fed03a2
Author: gyfora <gy...@gmail.com>
Authored: Sun Aug 3 15:51:46 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:50 2014 +0200
----------------------------------------------------------------------
.../streaming/connectors/rabbitmq/RMQSink.java | 2 +-
.../apache/flink/streaming/api/DataStream.java | 96 +++++++++++---------
.../flink/streaming/api/StreamConfig.java | 8 +-
.../api/StreamExecutionEnvironment.java | 52 ++++-------
.../api/function/co/CoMapFunction.java | 5 +-
.../api/function/sink/SinkFunction.java | 4 +-
.../api/function/source/SourceFunction.java | 4 +-
.../streaming/api/invokable/SinkInvokable.java | 11 +++
.../api/invokable/SourceInvokable.java | 14 ++-
.../api/invokable/StreamComponentInvokable.java | 15 ++-
.../operator/BatchReduceInvokable.java | 13 ++-
.../api/invokable/operator/FilterInvokable.java | 17 +++-
.../invokable/operator/FlatMapInvokable.java | 17 +++-
.../api/invokable/operator/MapInvokable.java | 17 +++-
.../operator/StreamReduceInvokable.java | 31 ++++++-
.../operator/WindowReduceInvokable.java | 13 ++-
.../invokable/operator/co/CoMapInvokable.java | 11 +++
.../api/streamcomponent/CoStreamTask.java | 12 ++-
.../api/streamcomponent/StreamSink.java | 2 +
.../api/streamcomponent/StreamSource.java | 2 +
.../api/streamcomponent/StreamTask.java | 2 +
.../util/serialization/FunctionTypeWrapper.java | 10 +-
.../apache/flink/streaming/api/IterateTest.java | 16 ++--
.../apache/flink/streaming/api/PrintTest.java | 2 +-
.../api/collector/DirectedOutputTest.java | 4 +-
.../api/invokable/operator/BatchReduceTest.java | 19 ++--
.../api/invokable/operator/FilterTest.java | 4 +-
.../api/invokable/operator/FlatMapTest.java | 8 +-
.../api/invokable/operator/MapTest.java | 12 +--
.../streamcomponent/StreamComponentTest.java | 10 +-
.../serialization/TypeSerializationTest.java | 6 +-
.../examples/basictopology/BasicTopology.java | 6 +-
.../examples/cellinfo/CellInfoLocal.java | 8 +-
.../examples/function/JSONParseFlatMap.java | 8 +-
.../flink/streaming/examples/join/JoinTask.java | 4 +-
.../ml/IncrementalLearningSkeleton.java | 8 +-
.../streaming/examples/ml/IncrementalOLS.java | 12 +--
.../examples/window/join/WindowJoinTask.java | 4 +-
.../examples/wordcount/WordCountCounter.java | 4 +-
.../examples/wordcount/WordCountSplitter.java | 4 +-
40 files changed, 299 insertions(+), 198 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index 4225cd3..c6f0ef5 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -32,7 +32,7 @@ import com.rabbitmq.client.ConnectionFactory;
public abstract class RMQSink<IN> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
- private static final Log LOG = LogFactory.getLog(RMQSource.class);
+ private static final Log LOG = LogFactory.getLog(RMQSink.class);
private boolean sendAndClose = false;
private boolean closeWithoutSend = false;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index d0f1294..f17dd1b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -24,11 +24,11 @@ import java.util.List;
import org.apache.commons.lang3.SerializationException;
import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.common.functions.AbstractFunction;
-import org.apache.flink.api.java.functions.FilterFunction;
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.functions.GroupReduceFunction;
-import org.apache.flink.api.java.functions.MapFunction;
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.java.functions.RichFilterFunction;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
@@ -124,6 +124,14 @@ public class DataStream<T> {
this.iterationID = dataStream.iterationID;
this.jobGraphBuilder = dataStream.jobGraphBuilder;
}
+
+
+ /**
+ * Partitioning strategy on the stream.
+ */
+ public static enum ConnectionType {
+ SHUFFLE, BROADCAST, FIELD, FORWARD, DISTRIBUTE
+ }
/**
* Initialize the connection and partitioning among the connected
@@ -133,7 +141,7 @@ public class DataStream<T> {
connectIDs = new ArrayList<String>();
connectIDs.add(getId());
partitioners = new ArrayList<StreamPartitioner<T>>();
- partitioners.add(new ShufflePartitioner<T>());
+ partitioners.add(new ForwardPartitioner<T>());
}
/**
@@ -203,7 +211,7 @@ public class DataStream<T> {
/**
* Gives the data transformation(vertex) a user defined name in order to use
- * at directed outputs. The {@link OutputSelector} of the input vertex
+ * with directed outputs. The {@link OutputSelector} of the input vertex
* should use this name for directed emits.
*
* @param name
@@ -312,7 +320,8 @@ public class DataStream<T> {
/**
* Sets the partitioning of the {@link DataStream} so that the output tuples
- * are forwarded to the local subtask of the next component.
+ * are forwarded to the local subtask of the next component. This is the
+ * default partitioner setting.
*
* @return The DataStream with shuffle partitioning set.
*/
@@ -342,19 +351,19 @@ public class DataStream<T> {
/**
* Applies a Map transformation on a {@link DataStream}. The transformation
- * calls a {@link MapFunction} for each element of the DataStream. Each
+ * calls a {@link RichMapFunction} for each element of the DataStream. Each
* MapFunction call returns exactly one element.
*
* @param mapper
- * The MapFunction that is called for each element of the
+ * The RichMapFunction that is called for each element of the
* DataStream.
* @param <R>
* output type
* @return The transformed DataStream.
*/
- public <R> StreamOperator<T, R> map(MapFunction<T, R> mapper) {
+ public <R> StreamOperator<T, R> map(RichMapFunction<T, R> mapper) {
return addFunction("map", mapper, new FunctionTypeWrapper<T, Tuple, R>(mapper,
- MapFunction.class, 0, -1, 1), new MapInvokable<T, R>(mapper));
+ RichMapFunction.class, 0, -1, 1), new MapInvokable<T, R>(mapper));
}
/**
@@ -372,8 +381,8 @@ public class DataStream<T> {
* {@link CoMapFunction#map2(Tuple)}
* @return The transformed DataStream
*/
- public <T2, R> DataStream<R> coMapWith(
- CoMapFunction<T, T2, R> coMapper, DataStream<T2> otherStream) {
+ public <T2, R> DataStream<R> coMapWith(CoMapFunction<T, T2, R> coMapper,
+ DataStream<T2> otherStream) {
return addCoFunction("coMap", new DataStream<T>(this), new DataStream<T2>(otherStream),
coMapper,
new FunctionTypeWrapper<T, T2, R>(coMapper, CoMapFunction.class, 0, 1, 2),
@@ -382,81 +391,82 @@ public class DataStream<T> {
/**
* Applies a FlatMap transformation on a {@link DataStream}. The
- * transformation calls a FlatMapFunction for each element of the
- * DataStream. Each FlatMapFunction call can return any number of elements
- * including none.
+ * transformation calls a {@link RichFlatMapFunction} for each element of
+ * the DataStream. Each RichFlatMapFunction call can return any number of
+ * elements including none.
*
* @param flatMapper
- * The FlatMapFunction that is called for each element of the
+ * The RichFlatMapFunction that is called for each element of the
* DataStream
*
* @param <R>
* output type
* @return The transformed DataStream.
*/
- public <R> StreamOperator<T, R> flatMap(FlatMapFunction<T, R> flatMapper) {
+ public <R> StreamOperator<T, R> flatMap(RichFlatMapFunction<T, R> flatMapper) {
return addFunction("flatMap", flatMapper, new FunctionTypeWrapper<T, Tuple, R>(flatMapper,
- FlatMapFunction.class, 0, -1, 1), new FlatMapInvokable<T, R>(flatMapper));
+ RichFlatMapFunction.class, 0, -1, 1), new FlatMapInvokable<T, R>(flatMapper));
}
/**
* Applies a Filter transformation on a {@link DataStream}. The
- * transformation calls a {@link FilterFunction} for each element of the
+ * transformation calls a {@link RichFilterFunction} for each element of the
* DataStream and retains only those element for which the function returns
* true. Elements for which the function returns false are filtered.
*
* @param filter
- * The FilterFunction that is called for each element of the
+ * The RichFilterFunction that is called for each element of the
* DataSet.
* @return The filtered DataStream.
*/
- public StreamOperator<T, T> filter(FilterFunction<T> filter) {
+ public StreamOperator<T, T> filter(RichFilterFunction<T> filter) {
return addFunction("filter", filter, new FunctionTypeWrapper<T, Tuple, T>(filter,
- FilterFunction.class, 0, -1, 0), new FilterInvokable<T>(filter));
+ RichFilterFunction.class, 0, -1, 0), new FilterInvokable<T>(filter));
}
/**
* Applies a reduce transformation on preset chunks of the DataStream. The
- * transformation calls a {@link GroupReduceFunction} for each tuple batch
- * of the predefined size. Each GroupReduceFunction call can return any
- * number of elements including none.
+ * transformation calls a {@link RichGroupReduceFunction} for each tuple
+ * batch of the predefined size. Each RichGroupReduceFunction call can
+ * return any number of elements including none.
*
*
* @param reducer
- * The GroupReduceFunction that is called for each tuple batch.
+ * The RichGroupReduceFunction that is called for each tuple
+ * batch.
* @param batchSize
* The number of tuples grouped together in the batch.
* @param <R>
* output type
* @return The modified DataStream.
*/
- public <R> StreamOperator<T, R> batchReduce(GroupReduceFunction<T, R> reducer,
- int batchSize) {
+ public <R> StreamOperator<T, R> batchReduce(RichGroupReduceFunction<T, R> reducer, int batchSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
- GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<T, R>(reducer,
+ RichGroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<T, R>(reducer,
batchSize));
}
/**
* Applies a reduce transformation on preset "time" chunks of the
- * DataStream. The transformation calls a {@link GroupReduceFunction} on
+ * DataStream. The transformation calls a {@link RichGroupReduceFunction} on
* records received during the predefined time window. The window shifted
- * after each reduce call. Each GroupReduceFunction call can return any
+ * after each reduce call. Each RichGroupReduceFunction call can return any
* number of elements including none.
*
*
* @param reducer
- * The GroupReduceFunction that is called for each time window.
+ * The RichGroupReduceFunction that is called for each time
+ * window.
* @param windowSize
* The time window to run the reducer on, in milliseconds.
* @param <R>
* output type
* @return The modified DataStream.
*/
- public <R> StreamOperator<T, R> windowReduce(GroupReduceFunction<T, R> reducer,
+ public <R> StreamOperator<T, R> windowReduce(RichGroupReduceFunction<T, R> reducer,
long windowSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
- GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<T, R>(reducer,
+ RichGroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<T, R>(reducer,
windowSize));
}
@@ -477,7 +487,7 @@ public class DataStream<T> {
* @return the data stream constructed
*/
private <R> StreamOperator<T, R> addFunction(String functionName,
- final AbstractFunction function, TypeSerializerWrapper<T, Tuple, R> typeWrapper,
+ final AbstractRichFunction function, TypeSerializerWrapper<T, Tuple, R> typeWrapper,
UserTaskInvokable<T, R> functionInvokable) {
DataStream<T> inputStream = new DataStream<T>(this);
@@ -500,9 +510,9 @@ public class DataStream<T> {
return returnStream;
}
- protected <T1, T2, R> DataStream<R> addCoFunction(
- String functionName, DataStream<T1> inputStream1, DataStream<T2> inputStream2,
- final AbstractFunction function, TypeSerializerWrapper<T1, T2, R> typeWrapper,
+ protected <T1, T2, R> DataStream<R> addCoFunction(String functionName,
+ DataStream<T1> inputStream1, DataStream<T2> inputStream2,
+ final AbstractRichFunction function, TypeSerializerWrapper<T1, T2, R> typeWrapper,
CoInvokable<T1, T2, R> functionInvokable) {
DataStream<R> returnStream = new DataStream<R>(environment, functionName);
@@ -535,7 +545,7 @@ public class DataStream<T> {
* @param typeNumber
* Number of the type (used at co-functions)
*/
- <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
+ private <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
for (int i = 0; i < inputStream.connectIDs.size(); i++) {
String inputID = inputStream.connectIDs.get(i);
StreamPartitioner<X> partitioner = inputStream.partitioners.get(i);
@@ -545,13 +555,13 @@ public class DataStream<T> {
}
/**
- * Adds the given sink to this environment. Only streams with sinks added
+ * Adds the given sink to this DataStream. Only streams with sinks added
* will be executed once the {@link StreamExecutionEnvironment#execute()}
* method is called.
*
* @param sinkFunction
* The object containing the sink's invoke function.
- * @return The modified DataStream.
+ * @return The closed DataStream.
*/
public DataStream<T> addSink(SinkFunction<T> sinkFunction) {
return addSink(new DataStream<T>(this), sinkFunction);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index a102a00..4572a84 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -23,7 +23,7 @@ import java.io.IOException;
import org.apache.commons.lang3.SerializationException;
import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.common.functions.AbstractRichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
@@ -257,13 +257,13 @@ public class StreamConfig {
return config.getInteger(INPUT_TYPE + inputNumber, 0);
}
- public void setFunctionClass(Class<? extends AbstractFunction> functionClass) {
+ public void setFunctionClass(Class<? extends AbstractRichFunction> functionClass) {
config.setClass("functionClass", functionClass);
}
@SuppressWarnings("unchecked")
- public Class<? extends AbstractFunction> getFunctionClass() {
- return (Class<? extends AbstractFunction>) config.getClass("functionClass", null);
+ public Class<? extends AbstractRichFunction> getFunctionClass() {
+ return (Class<? extends AbstractRichFunction>) config.getClass("functionClass", null);
}
@SuppressWarnings("unchecked")
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index 0e77912..76adf62 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -59,13 +59,6 @@ public abstract class StreamExecutionEnvironment {
protected JobGraphBuilder jobGraphBuilder;
- /**
- * Partitioning strategy on the stream.
- */
- public static enum ConnectionType {
- SHUFFLE, BROADCAST, FIELD, FORWARD, DISTRIBUTE
- }
-
// --------------------------------------------------------------------------------------------
// Constructor and Properties
// --------------------------------------------------------------------------------------------
@@ -113,15 +106,6 @@ public abstract class StreamExecutionEnvironment {
this.degreeOfParallelism = degreeOfParallelism;
}
- // protected void setMutability(DataStream<?> stream, boolean isMutable) {
- // jobGraphBuilder.setMutability(stream.getId(), isMutable);
- // }
- //
- // protected void setBufferTimeout(DataStream<?> stream, long bufferTimeout)
- // {
- // jobGraphBuilder.setBufferTimeout(stream.getId(), bufferTimeout);
- // }
-
/**
* Sets the number of hardware contexts (CPU cores / threads) used when
* executed in {@link LocalStreamEnvironment}.
@@ -186,17 +170,17 @@ public abstract class StreamExecutionEnvironment {
*
* @param data
* The collection of elements to create the DataStream from.
- * @param <X>
+ * @param <OUT>
* type of the returned stream
* @return The DataStream representing the elements.
*/
- public <X extends Serializable> DataStream<X> fromElements(X... data) {
- DataStream<X> returnStream = new DataStream<X>(this, "elements");
+ public <OUT extends Serializable> DataStream<OUT> fromElements(OUT... data) {
+ DataStream<OUT> returnStream = new DataStream<OUT>(this, "elements");
try {
- SourceFunction<X> function = new FromElementsFunction<X>(data);
- jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<X>(function),
- new ObjectTypeWrapper<X, Tuple, X>(data[0], null, data[0]), "source",
+ SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
+ jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
+ new ObjectTypeWrapper<OUT, Tuple, OUT>(data[0], null, data[0]), "source",
SerializationUtils.serialize(function), 1);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize elements");
@@ -212,25 +196,25 @@ public abstract class StreamExecutionEnvironment {
*
* @param data
* The collection of elements to create the DataStream from.
- * @param <X>
+ * @param <OUT>
* type of the returned stream
* @return The DataStream representing the elements.
*/
@SuppressWarnings("unchecked")
- public <X extends Serializable> DataStream<X> fromCollection(Collection<X> data) {
- DataStream<X> returnStream = new DataStream<X>(this, "elements");
+ public <OUT extends Serializable> DataStream<OUT> fromCollection(Collection<OUT> data) {
+ DataStream<OUT> returnStream = new DataStream<OUT>(this, "elements");
if (data.isEmpty()) {
throw new RuntimeException("Collection must not be empty");
}
try {
- SourceFunction<X> function = new FromElementsFunction<X>(data);
+ SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
jobGraphBuilder.addSource(
returnStream.getId(),
- new SourceInvokable<X>(new FromElementsFunction<X>(data)),
- new ObjectTypeWrapper<X, Tuple, X>((X) data.toArray()[0], null, (X) data
+ new SourceInvokable<OUT>(new FromElementsFunction<OUT>(data)),
+ new ObjectTypeWrapper<OUT, Tuple, OUT>((OUT) data.toArray()[0], null, (OUT) data
.toArray()[0]), "source", SerializationUtils.serialize(function), 1);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize collection");
@@ -259,16 +243,16 @@ public abstract class StreamExecutionEnvironment {
* the user defined function
* @param parallelism
* number of parallel instances of the function
- * @param <T>
+ * @param <OUT>
* type of the returned stream
* @return the data stream constructed
*/
- public <T> DataStream<T> addSource(SourceFunction<T> function, int parallelism) {
- DataStream<T> returnStream = new DataStream<T>(this, "source");
+ public <OUT> DataStream<OUT> addSource(SourceFunction<OUT> function, int parallelism) {
+ DataStream<OUT> returnStream = new DataStream<OUT>(this, "source");
try {
- jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<T>(function),
- new FunctionTypeWrapper<T, Tuple, T>(function, SourceFunction.class, 0, -1, 0),
+ jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
+ new FunctionTypeWrapper<OUT, Tuple, OUT>(function, SourceFunction.class, 0, -1, 0),
"source", SerializationUtils.serialize(function), parallelism);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize SourceFunction");
@@ -277,7 +261,7 @@ public abstract class StreamExecutionEnvironment {
return returnStream;
}
- public <T> DataStream<T> addSource(SourceFunction<T> sourceFunction) {
+ public <OUT> DataStream<OUT> addSource(SourceFunction<OUT> sourceFunction) {
return addSource(sourceFunction, 1);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
index 6e4d877..8404a80 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
@@ -18,9 +18,10 @@
package org.apache.flink.streaming.api.function.co;
-import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.common.functions.AbstractRichFunction;
-public abstract class CoMapFunction<IN1, IN2, OUT> extends AbstractFunction {
+
+public abstract class CoMapFunction<IN1, IN2, OUT> extends AbstractRichFunction {
private static final long serialVersionUID = 1L;
public abstract OUT map1(IN1 value);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
index 867c9f8..668837f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
@@ -21,9 +21,9 @@ package org.apache.flink.streaming.api.function.sink;
import java.io.Serializable;
-import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.common.functions.AbstractRichFunction;
-public abstract class SinkFunction<IN> extends AbstractFunction implements Serializable {
+public abstract class SinkFunction<IN> extends AbstractRichFunction implements Serializable {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
index 01d4dac..44e3387 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
@@ -19,10 +19,10 @@
package org.apache.flink.streaming.api.function.source;
-import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.common.functions.AbstractRichFunction;
import org.apache.flink.util.Collector;
-public abstract class SourceFunction<OUT> extends AbstractFunction {
+public abstract class SourceFunction<OUT> extends AbstractRichFunction {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index 92b1ea6..b733362 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.api.invokable;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
public class SinkInvokable<IN> extends StreamRecordInvokable<IN, IN> {
@@ -44,4 +45,14 @@ public class SinkInvokable<IN> extends StreamRecordInvokable<IN, IN> {
sinkFunction.invoke((IN) reuse.getObject());
}
}
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ sinkFunction.open(parameters);
+ }
+
+ @Override
+ public void close() throws Exception {
+ sinkFunction.close();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
index c7f0f09..d049bbf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
@@ -21,10 +21,10 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.function.source.SourceFunction;
-public class SourceInvokable<OUT> extends StreamComponentInvokable<OUT> implements
- Serializable {
+public class SourceInvokable<OUT> extends StreamComponentInvokable<OUT> implements Serializable {
private static final long serialVersionUID = 1L;
@@ -40,4 +40,14 @@ public class SourceInvokable<OUT> extends StreamComponentInvokable<OUT> implemen
public void invoke() throws Exception {
sourceFunction.invoke(collector);
}
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ sourceFunction.open(parameters);
+ }
+
+ @Override
+ public void close() throws Exception {
+ sourceFunction.close();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
index c011284..02ee5fd 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
@@ -21,9 +21,12 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.Collector;
-public abstract class StreamComponentInvokable<OUT> implements Serializable {
+public abstract class StreamComponentInvokable<OUT> extends AbstractRichFunction implements
+ Serializable {
private static final long serialVersionUID = 1L;
@@ -42,4 +45,14 @@ public abstract class StreamComponentInvokable<OUT> implements Serializable {
this.channelID = channelID;
}
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ System.out.println("Open not implemented");
+ }
+
+ @Override
+ public void close() throws Exception {
+ System.out.println("Close not implemented");
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 46e79de..61ba5a9 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -22,14 +22,13 @@ package org.apache.flink.streaming.api.invokable.operator;
import java.util.ArrayList;
import java.util.List;
-import org.apache.flink.api.java.functions.GroupReduceFunction;
+import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-public class BatchReduceInvokable<IN, OUT> extends
- StreamReduceInvokable<IN, OUT> {
+public class BatchReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private int batchSize;
- public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, int batchSize) {
+ public BatchReduceInvokable(RichGroupReduceFunction<IN, OUT> reduceFunction, int batchSize) {
this.reducer = reduceFunction;
this.batchSize = batchSize;
}
@@ -55,7 +54,7 @@ public class BatchReduceInvokable<IN, OUT> extends
tupleBatch.add(reuse.getObject());
resetReuse();
} while (counter < batchSize);
- reducer.reduce(tupleBatch.iterator(), collector);
+ reducer.reduce(tupleBatch, collector);
tupleBatch.clear();
counter = 0;
}
@@ -64,11 +63,11 @@ public class BatchReduceInvokable<IN, OUT> extends
@Override
protected void mutableInvoke() throws Exception {
- BatchIterator<IN> userIterator = new CounterIterator();
+ userIterator = new CounterIterator();
do {
if (userIterator.hasNext()) {
- reducer.reduce(userIterator, collector);
+ reducer.reduce(userIterable, collector);
userIterator.reset();
}
} while (reuse != null);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
index f2b2930..56ab680 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
@@ -19,16 +19,17 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.api.java.functions.FilterFunction;
+import org.apache.flink.api.java.functions.RichFilterFunction;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
private static final long serialVersionUID = 1L;
- FilterFunction<IN> filterFunction;
+ RichFilterFunction<IN> filterFunction;
- public FilterInvokable(FilterFunction<IN> filterFunction) {
+ public FilterInvokable(RichFilterFunction<IN> filterFunction) {
this.filterFunction = filterFunction;
}
@@ -50,4 +51,14 @@ public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
}
}
}
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ filterFunction.open(parameters);
+ }
+
+ @Override
+ public void close() throws Exception {
+ filterFunction.close();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
index 11e7853..7796230 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
@@ -19,15 +19,16 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.api.java.functions.FlatMapFunction;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
- private FlatMapFunction<IN, OUT> flatMapper;
+ private RichFlatMapFunction<IN, OUT> flatMapper;
- public FlatMapInvokable(FlatMapFunction<IN, OUT> flatMapper) {
+ public FlatMapInvokable(RichFlatMapFunction<IN, OUT> flatMapper) {
this.flatMapper = flatMapper;
}
@@ -45,4 +46,14 @@ public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
flatMapper.flatMap(reuse.getObject(), collector);
}
}
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ flatMapper.open(parameters);
+ }
+
+ @Override
+ public void close() throws Exception {
+ flatMapper.close();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
index 794d765..23fc31e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
@@ -19,15 +19,16 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.api.java.functions.MapFunction;
+import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
- private MapFunction<IN, OUT> mapper;
+ private RichMapFunction<IN, OUT> mapper;
- public MapInvokable(MapFunction<IN, OUT> mapper) {
+ public MapInvokable(RichMapFunction<IN, OUT> mapper) {
this.mapper = mapper;
}
@@ -45,4 +46,14 @@ public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
collector.collect(mapper.map(reuse.getObject()));
}
}
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ mapper.open(parameters);
+ }
+
+ @Override
+ public void close() throws Exception {
+ mapper.close();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
index a574ebc..1a402a1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
@@ -19,12 +19,35 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.api.java.functions.GroupReduceFunction;
+import java.util.Iterator;
+
+import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-public abstract class StreamReduceInvokable<IN, OUT> extends
- UserTaskInvokable<IN, OUT> {
+public abstract class StreamReduceInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
- protected GroupReduceFunction<IN, OUT> reducer;
+ protected RichGroupReduceFunction<IN, OUT> reducer;
+ protected BatchIterator<IN> userIterator;
+ protected BatchIterable userIterable;
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ userIterable = new BatchIterable();
+ reducer.open(parameters);
+ }
+
+ @Override
+ public void close() throws Exception {
+ reducer.close();
+ }
+
+ protected class BatchIterable implements Iterable<IN> {
+
+ @Override
+ public Iterator<IN> iterator() {
+ return userIterator;
+ }
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index 7710bd8..430a68e 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -22,16 +22,15 @@ package org.apache.flink.streaming.api.invokable.operator;
import java.util.ArrayList;
import java.util.List;
-import org.apache.flink.api.java.functions.GroupReduceFunction;
+import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-public class WindowReduceInvokable<IN, OUT> extends
- StreamReduceInvokable<IN, OUT> {
+public class WindowReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private long windowSize;
volatile boolean isRunning;
boolean window;
- public WindowReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize) {
+ public WindowReduceInvokable(RichGroupReduceFunction<IN, OUT> reduceFunction, long windowSize) {
this.reducer = reduceFunction;
this.windowSize = windowSize;
this.window = true;
@@ -56,7 +55,7 @@ public class WindowReduceInvokable<IN, OUT> extends
tupleBatch.add(reuse.getObject());
resetReuse();
} while (System.currentTimeMillis() - startTime < windowSize);
- reducer.reduce(tupleBatch.iterator(), collector);
+ reducer.reduce(tupleBatch, collector);
tupleBatch.clear();
startTime = System.currentTimeMillis();
}
@@ -64,11 +63,11 @@ public class WindowReduceInvokable<IN, OUT> extends
}
protected void mutableInvoke() throws Exception {
- BatchIterator<IN> userIterator = new WindowIterator();
+ userIterator = new WindowIterator();
do {
if (userIterator.hasNext()) {
- reducer.reduce(userIterator, collector);
+ reducer.reduce(userIterable, collector);
userIterator.reset();
}
} while (reuse != null);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
index e899367..ac71b22 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.api.invokable.operator.co;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
public class CoMapInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
@@ -53,4 +54,14 @@ public class CoMapInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
} while (!noMoreRecordOnInput1 && !noMoreRecordOnInput2);
}
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ mapper.open(parameters);
+ }
+
+ @Override
+ public void close() throws Exception {
+ mapper.close();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
index ede30b4..0e03915 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -25,7 +25,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.common.functions.AbstractRichFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
@@ -82,11 +82,13 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
}
@SuppressWarnings({ "unchecked", "rawtypes" })
- private void setDeserializers(Object function, Class<? extends AbstractFunction> clazz) {
- TypeInformation<IN1> inputTypeInfo1 = (TypeInformation<IN1>) typeWrapper.getInputTypeInfo1();
+ private void setDeserializers(Object function, Class<? extends AbstractRichFunction> clazz) {
+ TypeInformation<IN1> inputTypeInfo1 = (TypeInformation<IN1>) typeWrapper
+ .getInputTypeInfo1();
inputDeserializer1 = new StreamRecordSerializer<IN1>(inputTypeInfo1);
- TypeInformation<IN2> inputTypeInfo2 = (TypeInformation<IN2>) typeWrapper.getInputTypeInfo2();
+ TypeInformation<IN2> inputTypeInfo2 = (TypeInformation<IN2>) typeWrapper
+ .getInputTypeInfo2();
inputDeserializer2 = new StreamRecordSerializer(inputTypeInfo2);
}
@@ -154,7 +156,9 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
output.initializeSerializers();
}
+ userInvokable.open(getTaskConfiguration());
userInvokable.invoke();
+ userInvokable.close();
if (LOG.isDebugEnabled()) {
LOG.debug("TASK " + name + " invoke finished with instance id " + instanceID);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
index 2969e69..df95bda 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
@@ -58,7 +58,9 @@ public class StreamSink<IN> extends SingleInputAbstractStreamComponent<IN, IN> {
LOG.debug("SINK " + name + " invoked");
}
+ userInvokable.open(getTaskConfiguration());
userInvokable.invoke();
+ userInvokable.close();
if (LOG.isDebugEnabled()) {
LOG.debug("SINK " + name + " invoke finished");
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
index 70b8242..6644d6f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
@@ -73,7 +73,9 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
output.initializeSerializers();
}
+ userInvokable.open(getTaskConfiguration());
userInvokable.invoke();
+ userInvokable.close();
if (LOG.isDebugEnabled()) {
LOG.debug("SOURCE " + name + " invoke finished with instance id " + instanceID);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
index 3a3f9cf..f809dae 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
@@ -70,7 +70,9 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
output.initializeSerializers();
}
+ userInvokable.open(getTaskConfiguration());
userInvokable.invoke();
+ userInvokable.close();
if (LOG.isDebugEnabled()) {
LOG.debug("TASK " + name + " invoke finished with instance id " + instanceID);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
index 02a7554..54471ae 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
@@ -21,21 +21,21 @@ package org.apache.flink.streaming.util.serialization;
import java.io.IOException;
-import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.common.functions.AbstractRichFunction;
import org.apache.flink.api.java.typeutils.TypeExtractor;
public class FunctionTypeWrapper<IN1, IN2, OUT> extends
TypeSerializerWrapper<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
- private AbstractFunction function;
- private Class<? extends AbstractFunction> functionSuperClass;
+ private AbstractRichFunction function;
+ private Class<? extends AbstractRichFunction> functionSuperClass;
private int inTypeParameter1;
private int inTypeParameter2;
private int outTypeParameter;
- public FunctionTypeWrapper(AbstractFunction function,
- Class<? extends AbstractFunction> functionSuperClass, int inTypeParameter1,
+ public FunctionTypeWrapper(AbstractRichFunction function,
+ Class<? extends AbstractRichFunction> functionSuperClass, int inTypeParameter1,
int inTypeParameter2, int outTypeParameter) {
this.function = function;
this.functionSuperClass = functionSuperClass;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 1ea165f..68403a8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.List;
-import org.apache.flink.api.java.functions.FlatMapFunction;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.flink.util.Collector;
@@ -36,8 +36,7 @@ public class IterateTest {
private static final long MEMORYSIZE = 32;
private static boolean iterated = false;
- public static final class IterationHead extends
- FlatMapFunction<Boolean, Boolean> {
+ public static final class IterationHead extends RichFlatMapFunction<Boolean, Boolean> {
private static final long serialVersionUID = 1L;
@@ -53,8 +52,7 @@ public class IterateTest {
}
- public static final class IterationTail extends
- FlatMapFunction<Boolean,Boolean> {
+ public static final class IterationTail extends RichFlatMapFunction<Boolean, Boolean> {
private static final long serialVersionUID = 1L;
@@ -86,12 +84,12 @@ public class IterateTest {
for (int i = 0; i < 100000; i++) {
bl.add(false);
}
- DataStream<Boolean> source = env
- .fromCollection(bl);
+ DataStream<Boolean> source = env.fromCollection(bl);
IterativeDataStream<Boolean> iteration = source.iterate();
-
- DataStream<Boolean> increment = iteration.flatMap(new IterationHead()).flatMap(new IterationTail());
+
+ DataStream<Boolean> increment = iteration.flatMap(new IterationHead()).flatMap(
+ new IterationTail());
iteration.closeWith(increment).addSink(new MySink());
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
index 438887a..67dce9d 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
@@ -35,7 +35,7 @@ public class PrintTest {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
- env.generateSequence(1, 100000).print();
+ env.generateSequence(1, 10).print();
env.executeTest(MEMORYSIZE);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index e8742de..e0da783 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -25,7 +25,7 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
-import org.apache.flink.api.java.functions.MapFunction;
+import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
@@ -38,7 +38,7 @@ public class DirectedOutputTest {
static HashSet<Long> evenSet = new HashSet<Long>();
static HashSet<Long> oddSet = new HashSet<Long>();
- private static class PlusTwo extends MapFunction<Long, Long> {
+ private static class PlusTwo extends RichMapFunction<Long, Long> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
index d8c7213..c23c9a7 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -22,19 +22,18 @@ package org.apache.flink.streaming.api.invokable.operator;
import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
-import java.util.Iterator;
+import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.LogUtils;
-import org.junit.Test;
-import org.apache.flink.api.java.functions.GroupReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
import org.apache.log4j.Level;
+import org.junit.Test;
public class BatchReduceTest {
@@ -44,20 +43,20 @@ public class BatchReduceTest {
private static final long MEMORYSIZE = 32;
public static final class MyBatchReduce extends
- GroupReduceFunction<Tuple1<Double>, Tuple1<Double>> {
+ RichGroupReduceFunction<Tuple1<Double>, Tuple1<Double>> {
private static final long serialVersionUID = 1L;
@Override
- public void reduce(Iterator<Tuple1<Double>> values, Collector<Tuple1<Double>> out)
+ public void reduce(Iterable<Tuple1<Double>> values, Collector<Tuple1<Double>> out)
throws Exception {
Double sum = 0.;
Double count = 0.;
- while (values.hasNext()) {
- sum += values.next().f0;
+ for (Tuple1<Double> value : values) {
+ sum += value.f0;
count++;
}
- if(count>0){
+ if (count > 0) {
out.collect(new Tuple1<Double>(sum / count));
}
}
@@ -87,7 +86,7 @@ public class BatchReduceTest {
@Test
public void test() throws Exception {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
-
+
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(PARALlELISM);
@SuppressWarnings("unused")
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
index 3286ef1..2d4fe7a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
@@ -23,7 +23,7 @@ import java.io.Serializable;
import java.util.HashSet;
import java.util.Set;
-import org.apache.flink.api.java.functions.FilterFunction;
+import org.apache.flink.api.java.functions.RichFilterFunction;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
@@ -45,7 +45,7 @@ public class FilterTest implements Serializable {
}
}
- static class MyFilter extends FilterFunction<Integer> {
+ static class MyFilter extends RichFilterFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
index b299407..2a8973c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
@@ -25,7 +25,7 @@ import static org.junit.Assert.assertTrue;
import java.util.HashSet;
import java.util.Set;
-import org.apache.flink.api.java.functions.FlatMapFunction;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
@@ -37,7 +37,7 @@ import org.junit.Test;
public class FlatMapTest {
- public static final class MyFlatMap extends FlatMapFunction<Integer, Integer> {
+ public static final class MyFlatMap extends RichFlatMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@@ -49,7 +49,7 @@ public class FlatMapTest {
}
- public static final class ParallelFlatMap extends FlatMapFunction<Integer, Integer> {
+ public static final class ParallelFlatMap extends RichFlatMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -60,7 +60,7 @@ public class FlatMapTest {
}
- public static final class GenerateSequenceFlatMap extends FlatMapFunction<Long, Long> {
+ public static final class GenerateSequenceFlatMap extends RichFlatMapFunction<Long, Long> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
index 2a35de5..07478fb 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
@@ -27,7 +27,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
-import org.apache.flink.api.java.functions.MapFunction;
+import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
@@ -84,7 +84,7 @@ public class MapTest {
}
}
- public static final class MyMap extends MapFunction<Integer, Integer> {
+ public static final class MyMap extends RichMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -94,7 +94,7 @@ public class MapTest {
}
}
- public static final class MySingleJoinMap extends MapFunction<Integer, Integer> {
+ public static final class MySingleJoinMap extends RichMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -104,7 +104,7 @@ public class MapTest {
}
}
- public static final class MyMultipleJoinMap extends MapFunction<Integer, Integer> {
+ public static final class MyMultipleJoinMap extends RichMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -114,7 +114,7 @@ public class MapTest {
}
}
- public static final class MyFieldsMap extends MapFunction<Integer, Integer> {
+ public static final class MyFieldsMap extends RichMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
private int counter = 0;
@@ -128,7 +128,7 @@ public class MapTest {
}
}
- public static final class MyDiffFieldsMap extends MapFunction<Integer, Integer> {
+ public static final class MyDiffFieldsMap extends RichMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
private int counter = 0;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
index 89a2c7c..566acd2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
@@ -24,17 +24,17 @@ import static org.junit.Assert.assertEquals;
import java.util.HashMap;
import java.util.Map;
+import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.util.Collector;
import org.junit.BeforeClass;
import org.junit.Test;
-import org.apache.flink.api.java.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
public class StreamComponentTest {
@@ -59,7 +59,7 @@ public class StreamComponentTest {
}
}
- public static class MyTask extends MapFunction<Tuple1<Integer>, Tuple2<Integer, Integer>> {
+ public static class MyTask extends RichMapFunction<Tuple1<Integer>, Tuple2<Integer, Integer>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
index 4ca191e..7347f62 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
@@ -25,14 +25,14 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.java.functions.MapFunction;
+import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple1;
import org.junit.Test;
public class TypeSerializationTest {
- private static class MyMap extends MapFunction<Tuple1<Integer>, Tuple1<String>> {
+ private static class MyMap extends RichMapFunction<Tuple1<Integer>, Tuple1<String>> {
private static final long serialVersionUID = 1L;
@Override
@@ -45,7 +45,7 @@ public class TypeSerializationTest {
@Test
public void functionTypeSerializationTest() {
TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>> ser = new FunctionTypeWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>>(
- new MyMap(), MapFunction.class, 0, -1, 1);
+ new MyMap(), RichMapFunction.class, 0, -1, 1);
byte[] serializedType = SerializationUtils.serialize(ser);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
index 0bb722c..04b8ee2 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
@@ -19,11 +19,11 @@
package org.apache.flink.streaming.examples.basictopology;
+import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.api.java.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
public class BasicTopology {
@@ -42,7 +42,7 @@ public class BasicTopology {
}
}
- public static class BasicMap extends MapFunction<Tuple1<String>, Tuple1<String>> {
+ public static class BasicMap extends RichMapFunction<Tuple1<String>, Tuple1<String>> {
private static final long serialVersionUID = 1L;
// map to the same tuple
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
index 9c0638d..5704723 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
@@ -21,12 +21,12 @@ package org.apache.flink.streaming.examples.cellinfo;
import java.util.Random;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.util.Collector;
public class CellInfoLocal {
@@ -77,7 +77,7 @@ public class CellInfoLocal {
}
private final static class CellTask extends
- FlatMapFunction<Tuple4<Boolean, Integer, Long, Integer>, Tuple1<String>> {
+ RichFlatMapFunction<Tuple4<Boolean, Integer, Long, Integer>, Tuple1<String>> {
private static final long serialVersionUID = 1L;
private WorkerEngineExact engine = new WorkerEngineExact(10, 500,
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
index 267d035..9a3672e 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
@@ -21,8 +21,8 @@ package org.apache.flink.streaming.examples.function;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.sling.commons.json.JSONException;
import org.apache.sling.commons.json.JSONObject;
@@ -34,10 +34,10 @@ import org.apache.sling.commons.json.JSONObject;
* Type of the returned elements.
*/
public abstract class JSONParseFlatMap<IN, OUT> extends
- FlatMapFunction<IN, OUT> {
+ RichFlatMapFunction<IN, OUT> {
private static final long serialVersionUID = 1L;
- private static final Log LOG = LogFactory.getLog(DataStream.class);
+ private static final Log LOG = LogFactory.getLog(JSONParseFlatMap.class);
/**
* Get the value of a field in a JSON text.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinTask.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinTask.java
index 8287904..c4414d9 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinTask.java
@@ -22,12 +22,12 @@ package org.apache.flink.streaming.examples.join;
import java.util.ArrayList;
import java.util.HashMap;
-import org.apache.flink.api.java.functions.FlatMapFunction;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.util.Collector;
public class JoinTask extends
- FlatMapFunction<Tuple3<String, String, Integer>, Tuple3<String, Integer, Integer>> {
+ RichFlatMapFunction<Tuple3<String, String, Integer>, Tuple3<String, Integer, Integer>> {
private static final long serialVersionUID = 749913336259789039L;
private HashMap<String, ArrayList<Integer>> gradeHashmap;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index d855639..05a7eb1 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -19,11 +19,11 @@
package org.apache.flink.streaming.examples.ml;
+import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.api.java.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
public class IncrementalLearningSkeleton {
@@ -69,7 +69,7 @@ public class IncrementalLearningSkeleton {
}
// Task for building up-to-date partial models on new training data
- public static class PartialModelBuilder extends MapFunction<Tuple1<Integer>, Tuple1<Integer>> {
+ public static class PartialModelBuilder extends RichMapFunction<Tuple1<Integer>, Tuple1<Integer>> {
private static final long serialVersionUID = 1L;
@Override
@@ -86,7 +86,7 @@ public class IncrementalLearningSkeleton {
// Task for performing prediction using the model produced in
// batch-processing and the up-to-date partial model
- public static class Predictor extends MapFunction<Tuple1<Integer>, Tuple1<Integer>> {
+ public static class Predictor extends RichMapFunction<Tuple1<Integer>, Tuple1<Integer>> {
private static final long serialVersionUID = 1L;
Tuple1<Integer> batchModel = null;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
index b4c97c3..1b1cfe1 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
@@ -23,13 +23,13 @@ import java.util.Random;
import org.apache.commons.lang.ArrayUtils;
import org.apache.commons.math.stat.regression.OLSMultipleLinearRegression;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.api.java.functions.MapFunction;
+import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
public class IncrementalOLS {
@@ -78,7 +78,7 @@ public class IncrementalOLS {
}
public static class PartialModelBuilder extends
- MapFunction<Tuple2<Double, Double[]>, Tuple2<Boolean, Double[]>> {
+ RichMapFunction<Tuple2<Double, Double[]>, Tuple2<Boolean, Double[]>> {
private static final long serialVersionUID = 1L;
@Override
@@ -116,7 +116,7 @@ public class IncrementalOLS {
}
// TODO: How do I know the x for which I have predicted y?
- public static class Predictor extends MapFunction<Tuple2<Boolean, Double[]>, Tuple1<Double>> {
+ public static class Predictor extends RichMapFunction<Tuple2<Boolean, Double[]>, Tuple1<Double>> {
private static final long serialVersionUID = 1L;
// StreamRecord batchModel = null;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinTask.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinTask.java
index a0ed3a4..bf536f4 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinTask.java
@@ -23,13 +23,13 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
-import org.apache.flink.api.java.functions.FlatMapFunction;
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.util.Collector;
public class WindowJoinTask extends
- FlatMapFunction<Tuple4<String, String, Integer, Long>, Tuple3<String, Integer, Integer>> {
+ RichFlatMapFunction<Tuple4<String, String, Integer, Long>, Tuple3<String, Integer, Integer>> {
class SalaryProgress {
public SalaryProgress(Integer salary, Long progress) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/776bd3f6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
index 53c23d6..7142b58 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
@@ -22,10 +22,10 @@ package org.apache.flink.streaming.examples.wordcount;
import java.util.HashMap;
import java.util.Map;
-import org.apache.flink.api.java.functions.MapFunction;
+import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple2;
-public class WordCountCounter extends MapFunction<String, Tuple2<String, Integer>> {
+public class WordCountCounter extends RichMapFunction<String, Tuple2<String, Integer>> {
private static final long serialVersionUID = 1L;
private Map<String, Integer> wordCounts = new HashMap<String, Integer>();
[18/51] [abbrv] git commit: [streaming] Javadocs for connectors
Posted by se...@apache.org.
[streaming] Javadocs for connectors
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/2f704aeb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/2f704aeb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/2f704aeb
Branch: refs/heads/master
Commit: 2f704aeb3eb94a7b24641408699d54cd40a4d8cb
Parents: 7c5bc3c
Author: jfeher <fe...@gmail.com>
Authored: Fri Aug 1 11:22:31 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:21:13 2014 +0200
----------------------------------------------------------------------
.../streaming/connectors/flume/FlumeSink.java | 51 +++++++++++++++---
.../streaming/connectors/flume/FlumeSource.java | 57 ++++++++++++++++++--
.../connectors/flume/FlumeTopology.java | 2 +-
.../streaming/connectors/kafka/KafkaSink.java | 22 ++++++++
.../streaming/connectors/kafka/KafkaSource.java | 36 +++++++++----
.../streaming/connectors/rabbitmq/RMQSink.java | 29 +++++++++-
.../connectors/rabbitmq/RMQSource.java | 27 +++++++++-
7 files changed, 201 insertions(+), 23 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2f704aeb/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
index 3379fcb..20a3a4a 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
@@ -21,21 +21,22 @@ package org.apache.flink.streaming.connectors.flume;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.connectors.rabbitmq.RMQSource;
import org.apache.flume.Event;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.FlumeException;
import org.apache.flume.api.RpcClient;
import org.apache.flume.api.RpcClientFactory;
import org.apache.flume.event.EventBuilder;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.connectors.rabbitmq.RMQSource;
public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
private static final Log LOG = LogFactory.getLog(RMQSource.class);
-
+
private transient FlinkRpcClientFacade client;
boolean initDone = false;
String host;
@@ -48,6 +49,13 @@ public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
this.port = port;
}
+ /**
+ * Receives tuples from the Apache Flink {@link DataStream} and forwards them to
+ * Apache Flume.
+ *
+ * @param tuple
+ * The tuple arriving from the datastream
+ */
@Override
public void invoke(IN tuple) {
@@ -66,6 +74,13 @@ public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
}
+ /**
+ * Serializes tuples into byte arrays.
+ *
+ * @param tuple
+ * The tuple used for the serialization
+ * @return The serialized byte array.
+ */
public abstract byte[] serialize(IN tuple);
private class FlinkRpcClientFacade {
@@ -73,6 +88,14 @@ public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
private String hostname;
private int port;
+ /**
+ * Initializes the connection to Apache Flume.
+ *
+ * @param hostname
+ * The host
+ * @param port
+ * The port.
+ */
public void init(String hostname, int port) {
// Setup the RPC connection
this.hostname = hostname;
@@ -80,12 +103,13 @@ public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
int initCounter = 0;
while (true) {
if (initCounter >= 90) {
- System.exit(1);
+ new RuntimeException("Cannot establish connection with" + port + " at " + host);
}
try {
this.client = RpcClientFactory.getDefaultInstance(hostname, port);
} catch (FlumeException e) {
- // Wait one second if the connection failed before the next try
+ // Wait one second if the connection failed before the next
+ // try
try {
Thread.sleep(1000);
} catch (InterruptedException e1) {
@@ -102,6 +126,12 @@ public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
initDone = true;
}
+ /**
+ * Sends byte arrays as {@link Event} series to Apache Flume.
+ *
+ * @param data
+ * The byte array to send to Apache FLume
+ */
public void sendDataToFlume(byte[] data) {
Event event = EventBuilder.withBody(data);
@@ -116,16 +146,25 @@ public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
}
}
+ /**
+ * Closes the RpcClient.
+ */
public void close() {
client.close();
}
}
+ /**
+ * Closes the connection only when the next message is sent after this call.
+ */
public void sendAndClose() {
sendAndClose = true;
}
+ /**
+ * Closes the connection immediately and no further data will be sent.
+ */
public void closeWithoutSend() {
client.close();
closeWithoutSend = true;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2f704aeb/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
index 5b90e14..8b102a8 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
@@ -26,8 +26,8 @@ import org.apache.flume.channel.ChannelProcessor;
import org.apache.flume.source.AvroSource;
import org.apache.flume.source.avro.AvroFlumeEvent;
import org.apache.flume.source.avro.Status;
-
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
@@ -45,12 +45,28 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
public class MyAvroSource extends AvroSource {
Collector<IN> collector;
+ /**
+ * Sends the AvroFlumeEvent from it's argument list to the Apache Flink
+ * {@link DataStream}.
+ *
+ * @param avroEvent
+ * The event that should be sent to the dataStream
+ * @return A {@link Status}.OK message if sending the event was successful.
+ */
@Override
public Status append(AvroFlumeEvent avroEvent) {
collect(avroEvent);
return Status.OK;
}
+ /**
+ * Sends the AvroFlumeEvents from it's argument list to the Apache Flink
+ * {@link DataStream}.
+ *
+ * @param events
+ * The events that is sent to the dataStream
+ * @return A Status.OK message if sending the events was successful.
+ */
@Override
public Status appendBatch(List<AvroFlumeEvent> events) {
for (AvroFlumeEvent avroEvent : events) {
@@ -60,6 +76,13 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
return Status.OK;
}
+ /**
+ * Deserializes the AvroFlumeEvent before sending it to the Apache Flink
+ * {@link DataStream}.
+ *
+ * @param avroEvent
+ * The event that is sent to the dataStream
+ */
private void collect(AvroFlumeEvent avroEvent) {
byte[] b = avroEvent.getBody().array();
IN tuple = FlumeSource.this.deserialize(b);
@@ -78,8 +101,22 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
private boolean sendAndClose = false;
private volatile boolean sendDone = false;
- public abstract IN deserialize(byte[] msg);
-
+ /**
+ * Deserializes the incoming data.
+ *
+ * @param message
+ * The incoming message in a byte array
+ * @return The deserialized message in the required format.
+ */
+ public abstract IN deserialize(byte[] message);
+
+ /**
+ * Configures the AvroSource. Also sets the collector so the application can
+ * use it from outside of the invoke function.
+ *
+ * @param collector
+ * The collector used in the invoke function
+ */
public void configureAvroSource(Collector<IN> collector) {
avroSource = new MyAvroSource();
@@ -88,10 +125,18 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
context.put("port", port);
context.put("bind", host);
avroSource.configure(context);
+ // An instance of a ChannelProcessor is required for configuring the
+ // avroSource although it will not be used in this case.
ChannelProcessor cp = new ChannelProcessor(null);
avroSource.setChannelProcessor(cp);
}
+ /**
+ * Configures the AvroSource and runs until the user calls a close function.
+ *
+ * @param collector
+ * The Collector for sending data to the datastream
+ */
@Override
public void invoke(Collector<IN> collector) throws Exception {
configureAvroSource(collector);
@@ -104,10 +149,16 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
avroSource.stop();
}
+ /**
+ * Closes the connection only when the next message is sent after this call.
+ */
public void sendAndClose() {
sendAndClose = true;
}
+ /**
+ * Closes the connection immediately and no further data will be sent.
+ */
public void closeWithoutSend() {
closeWithoutSend = true;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2f704aeb/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
index 779a5fb..3c45cd4 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
@@ -39,7 +39,7 @@ public class FlumeTopology {
try {
sendAndClose();
} catch (Exception e) {
- new RuntimeException("Error while closing RMQ connection with " + port + " at "
+ new RuntimeException("Error while closing Flume connection with " + port + " at "
+ host, e);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2f704aeb/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
index 44867ef..7e3f3db 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
@@ -45,6 +45,9 @@ public abstract class KafkaSink<IN extends Tuple, OUT> extends SinkFunction<IN>
}
+ /**
+ * Initializes the connection to Kafka.
+ */
public void initialize() {
props = new Properties();
@@ -57,6 +60,12 @@ public abstract class KafkaSink<IN extends Tuple, OUT> extends SinkFunction<IN>
initDone = true;
}
+ /**
+ * Called when new data arrives to the sink, and forwards it to Kafka.
+ *
+ * @param tuple
+ * The incoming data
+ */
@Override
public void invoke(IN tuple) {
if (!initDone) {
@@ -75,13 +84,26 @@ public abstract class KafkaSink<IN extends Tuple, OUT> extends SinkFunction<IN>
}
}
+ /**
+ * Serializes tuples into byte arrays.
+ *
+ * @param tuple
+ * The tuple used for the serialization
+ * @return The serialized byte array.
+ */
public abstract OUT serialize(IN tuple);
+ /**
+ * Closes the connection immediately and no further data will be sent.
+ */
public void closeWithoutSend() {
producer.close();
closeWithoutSend = true;
}
+ /**
+ * Closes the connection only when the next message is sent after this call.
+ */
public void sendAndClose() {
sendAndClose = true;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2f704aeb/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
index 32b11d9..623e3b8 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
@@ -28,14 +28,12 @@ import kafka.consumer.ConsumerConfig;
import kafka.consumer.ConsumerIterator;
import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
+
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
-/**
- * Source for reading messages from a Kafka queue. The source currently only
- * support string messages.
- */
public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
private static final long serialVersionUID = 1L;
@@ -49,14 +47,16 @@ public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
IN outTuple;
- public KafkaSource(String zkQuorum, String groupId, String topicId,
- int numThreads) {
+ public KafkaSource(String zkQuorum, String groupId, String topicId, int numThreads) {
this.zkQuorum = zkQuorum;
this.groupId = groupId;
this.topicId = topicId;
this.numThreads = numThreads;
}
+ /**
+ * Initializes the connection to Kafka.
+ */
private void initializeConnection() {
Properties props = new Properties();
props.put("zookeeper.connect", zkQuorum);
@@ -64,10 +64,15 @@ public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
props.put("zookeeper.session.timeout.ms", "400");
props.put("zookeeper.sync.time.ms", "200");
props.put("auto.commit.interval.ms", "1000");
- consumer = kafka.consumer.Consumer
- .createJavaConsumerConnector(new ConsumerConfig(props));
+ consumer = kafka.consumer.Consumer.createJavaConsumerConnector(new ConsumerConfig(props));
}
+ /**
+ * Called to forward the data from the source to the {@link DataStream}.
+ *
+ * @param collector
+ * The Collector for sending data to the dataStream
+ */
@Override
public void invoke(Collector<IN> collector) throws Exception {
initializeConnection();
@@ -92,12 +97,25 @@ public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
consumer.shutdown();
}
- public abstract IN deserialize(byte[] msg);
+ /**
+ * Deserializes the incoming data.
+ *
+ * @param message
+ * The incoming message in a byte array
+ * @return The deserialized message in the required format.
+ */
+ public abstract IN deserialize(byte[] message);
+ /**
+ * Closes the connection immediately and no further data will be sent.
+ */
public void closeWithoutSend() {
closeWithoutSend = true;
}
+ /**
+ * Closes the connection only when the next message is sent after this call.
+ */
public void sendAndClose() {
sendAndClose = true;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2f704aeb/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index 663fc13..d7ed17a 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -50,6 +50,9 @@ public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
this.QUEUE_NAME = QUEUE_NAME;
}
+ /**
+ * Initializes the connection to RMQ.
+ */
public void initializeConnection() {
factory = new ConnectionFactory();
factory.setHost(HOST_NAME);
@@ -64,6 +67,12 @@ public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
initDone = true;
}
+ /**
+ * Called when new data arrives to the sink, and forwards it to RMQ.
+ *
+ * @param tuple
+ * The incoming data
+ */
@Override
public void invoke(IN tuple) {
if (!initDone) {
@@ -87,8 +96,18 @@ public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
}
}
- public abstract byte[] serialize(Tuple t);
-
+ /**
+ * Serializes tuples into byte arrays.
+ *
+ * @param tuple
+ * The tuple used for the serialization
+ * @return The serialized byte array.
+ */
+ public abstract byte[] serialize(Tuple tuple);
+
+ /**
+ * Closes the connection.
+ */
private void closeChannel() {
try {
channel.close();
@@ -100,11 +119,17 @@ public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
}
+ /**
+ * Closes the connection immediately and no further data will be sent.
+ */
public void closeWithoutSend() {
closeChannel();
closeWithoutSend = true;
}
+ /**
+ * Closes the connection only when the next message is sent after this call.
+ */
public void sendAndClose() {
sendAndClose = true;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2f704aeb/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index 4fd2235..dfea55a 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -30,6 +30,7 @@ import com.rabbitmq.client.ConnectionFactory;
import com.rabbitmq.client.QueueingConsumer;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
@@ -56,6 +57,9 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
this.QUEUE_NAME = QUEUE_NAME;
}
+ /**
+ * Initializes the connection to RMQ.
+ */
private void initializeConnection() {
factory = new ConnectionFactory();
factory.setHost(HOST_NAME);
@@ -71,6 +75,12 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
}
}
+ /**
+ * Called to forward the data from the source to the {@link DataStream}.
+ *
+ * @param collector
+ * The Collector for sending data to the dataStream
+ */
@Override
public void invoke(Collector<IN> collector) throws Exception {
initializeConnection();
@@ -105,12 +115,25 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
}
- public abstract IN deserialize(byte[] t);
-
+ /**
+ * Deserializes the incoming data.
+ *
+ * @param message
+ * The incoming message in a byte array
+ * @return The deserialized message in the required format.
+ */
+ public abstract IN deserialize(byte[] message);
+
+ /**
+ * Closes the connection immediately and no further data will be sent.
+ */
public void closeWithoutSend() {
closeWithoutSend = true;
}
+ /**
+ * Closes the connection only when the next message is sent after this call.
+ */
public void sendAndClose() {
sendAndClose = true;
}
[48/51] [abbrv] git commit: [streaming] Several minor bugfixes
Posted by se...@apache.org.
[streaming] Several minor bugfixes
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/e6766fd5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/e6766fd5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/e6766fd5
Branch: refs/heads/master
Commit: e6766fd5f2dc9ef6a3c1b055f7cf0acc655c039f
Parents: fc46d4c
Author: gyfora <gy...@gmail.com>
Authored: Sat Aug 9 22:56:16 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:43 2014 +0200
----------------------------------------------------------------------
.../streaming/connectors/flume/FlumeSink.java | 2 +-
.../connectors/flume/FlumeTopology.java | 2 +-
.../streaming/connectors/kafka/KafkaSink.java | 2 +-
.../streaming/connectors/rabbitmq/RMQSink.java | 2 +-
.../connectors/rabbitmq/RMQSource.java | 4 +-
.../connectors/twitter/TwitterSource.java | 4 +-
.../flink/streaming/api/StreamConfig.java | 3 +-
.../streaming/api/datastream/DataStream.java | 1 +
.../environment/StreamExecutionEnvironment.java | 1 +
.../api/function/source/FileSourceFunction.java | 2 +-
.../api/function/source/FileStreamFunction.java | 2 +-
.../function/source/FromElementsFunction.java | 1 +
.../api/invokable/operator/co/CoInvokable.java | 1 -
.../AbstractStreamComponent.java | 48 ++++++++++----------
.../streamcomponent/StreamIterationSink.java | 2 -
.../flink/streaming/api/streamrecord/UID.java | 12 +++--
.../partitioner/FieldsPartitioner.java | 4 +-
.../streaming/state/SlidingWindowState.java | 4 +-
.../streaming/state/InternalStateTest.java | 13 +-----
.../streaming/state/SlidingWindowStateTest.java | 1 -
.../CollaborativeFilteringSource.java | 2 +-
.../iterative/pagerank/PageRankSource.java | 2 +-
.../examples/iterative/sssp/SSSPSource.java | 2 +-
23 files changed, 53 insertions(+), 64 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
index 349a9a5..7ffd9e6 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
@@ -101,7 +101,7 @@ public abstract class FlumeSink<IN> implements SinkFunction<IN> {
int initCounter = 0;
while (true) {
if (initCounter >= 90) {
- new RuntimeException("Cannot establish connection with" + port + " at " + host);
+ throw new RuntimeException("Cannot establish connection with" + port + " at " + host);
}
try {
this.client = RpcClientFactory.getDefaultInstance(hostname, port);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
index d6e7570..21cca56 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
@@ -38,7 +38,7 @@ public class FlumeTopology {
try {
sendAndClose();
} catch (Exception e) {
- new RuntimeException("Error while closing Flume connection with " + port + " at "
+ throw new RuntimeException("Error while closing Flume connection with " + port + " at "
+ host, e);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
index 183860e..89801bb 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
@@ -31,7 +31,7 @@ public abstract class KafkaSink<IN, OUT> implements SinkFunction<IN> {
private static final long serialVersionUID = 1L;
private kafka.javaapi.producer.Producer<Integer, OUT> producer;
- static Properties props;
+ private Properties props;
private String topicId;
private String brokerAddr;
private boolean sendAndClose = false;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index 2e3a8a6..12b6341 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -112,7 +112,7 @@ public abstract class RMQSink<IN> implements SinkFunction<IN> {
channel.close();
connection.close();
} catch (IOException e) {
- new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME + " at "
+ throw new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME + " at "
+ HOST_NAME, e);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index 19ea57f..dc38977 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -68,7 +68,7 @@ public abstract class RMQSource<OUT> extends RichSourceFunction<OUT> {
consumer = new QueueingConsumer(channel);
channel.basicConsume(QUEUE_NAME, true, consumer);
} catch (IOException e) {
- new RuntimeException("Cannot create RMQ connection with " + QUEUE_NAME + " at "
+ throw new RuntimeException("Cannot create RMQ connection with " + QUEUE_NAME + " at "
+ HOST_NAME, e);
}
}
@@ -107,7 +107,7 @@ public abstract class RMQSource<OUT> extends RichSourceFunction<OUT> {
try {
connection.close();
} catch (IOException e) {
- new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME + " at "
+ throw new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME + " at "
+ HOST_NAME, e);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
index 17e3b02..a111b98 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
@@ -145,7 +145,7 @@ public class TwitterSource implements SourceFunction<String> {
properties.load(input);
input.close();
} catch (IOException ioe) {
- new RuntimeException("Cannot open .properties file: " + authPath,
+ throw new RuntimeException("Cannot open .properties file: " + authPath,
ioe);
}
return properties;
@@ -221,7 +221,7 @@ public class TwitterSource implements SourceFunction<String> {
}
}
} catch (InterruptedException e) {
- new RuntimeException("'Waiting for tweet' thread is interrupted", e);
+ throw new RuntimeException("'Waiting for tweet' thread is interrupted", e);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index da1189e..3d124c2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -129,9 +129,8 @@ public class StreamConfig {
try {
return deserializeObject(config.getBytes(SERIALIZEDUDF, null));
} catch (Exception e) {
- new StreamComponentException("Cannot instantiate user function");
+ throw new StreamComponentException("Cannot instantiate user function");
}
- return null;
}
public void setComponentName(String componentName) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index ab14bc6..243cc5e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -159,6 +159,7 @@ public abstract class DataStream<OUT> {
* The DataStreams to merge output with.
* @return The {@link MergedDataStream}.
*/
+ @SuppressWarnings("unchecked")
public MergedDataStream<OUT> merge(DataStream<OUT>... streams) {
MergedDataStream<OUT> returnStream = new MergedDataStream<OUT>(this);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 68e2421..9b74b40 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -194,6 +194,7 @@ public abstract class StreamExecutionEnvironment {
* type of the returned stream
* @return The DataStream representing the elements.
*/
+ @SuppressWarnings("unchecked")
public <OUT extends Serializable> DataStreamSource<OUT> fromElements(OUT... data) {
DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements");
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
index 6c8cd3a..b4dac07 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
@@ -39,7 +39,7 @@ public class FileSourceFunction implements SourceFunction<String> {
BufferedReader br = new BufferedReader(new FileReader(path));
String line = br.readLine();
while (line != null) {
- if (line != "") {
+ if (!line.equals("")) {
collector.collect(line);
}
line = br.readLine();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
index 799e700..0cdd2b6 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
@@ -40,7 +40,7 @@ public class FileStreamFunction implements SourceFunction<String> {
BufferedReader br = new BufferedReader(new FileReader(path));
String line = br.readLine();
while (line != null) {
- if (line != "") {
+ if (!line.equals("")) {
collector.collect(line);
}
line = br.readLine();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
index 98e012b..b115af2 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
@@ -29,6 +29,7 @@ public class FromElementsFunction<T> implements SourceFunction<T> {
Iterable<T> iterable;
+ @SuppressWarnings("unchecked")
public FromElementsFunction(T... elements) {
this.iterable = Arrays.asList(elements);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
index 4ac75a1..1e8623c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
@@ -35,7 +35,6 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends
private static final long serialVersionUID = 1L;
- protected Collector<OUT> collector;
protected MutableObjectIterator<StreamRecord<IN1>> recordIterator1;
protected MutableObjectIterator<StreamRecord<IN2>> recordIterator2;
protected StreamRecord<IN1> reuse1;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index 324b5ba..c973894 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -131,31 +131,31 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
try {
outputPartitioner = configuration.getPartitioner(outputNumber);
- RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output;
-
- if (bufferTimeout > 0) {
- output = new StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>>(this,
- outputPartitioner, bufferTimeout);
- } else {
- output = new RecordWriter<SerializationDelegate<StreamRecord<OUT>>>(this,
- outputPartitioner);
- }
-
- outputs.add(output);
- List<String> outputName = configuration.getOutputName(outputNumber);
-
- if (collector != null) {
- collector.addOutput(output, outputName);
- }
-
- if (LOG.isTraceEnabled()) {
- LOG.trace("Partitioner set: " + outputPartitioner.getClass().getSimpleName()
- + " with " + outputNumber + " outputs");
- }
} catch (Exception e) {
- throw new StreamComponentException("Cannot deserialize partitioner "
- + outputPartitioner.getClass().getSimpleName() + " of " + name + " with "
- + outputNumber + " outputs", e);
+ throw new StreamComponentException("Cannot deserialize partitioner for " + name
+ + " with " + outputNumber + " outputs", e);
+ }
+
+ RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output;
+
+ if (bufferTimeout > 0) {
+ output = new StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>>(this,
+ outputPartitioner, bufferTimeout);
+ } else {
+ output = new RecordWriter<SerializationDelegate<StreamRecord<OUT>>>(this,
+ outputPartitioner);
+ }
+
+ outputs.add(output);
+ List<String> outputName = configuration.getOutputName(outputNumber);
+
+ if (collector != null) {
+ collector.addOutput(output, outputName);
+ }
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Partitioner set: " + outputPartitioner.getClass().getSimpleName() + " with "
+ + outputNumber + " outputs");
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
index 5586887..069e846 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
@@ -26,7 +26,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.util.MutableObjectIterator;
import org.apache.flink.util.StringUtils;
public class StreamIterationSink<IN extends Tuple> extends
@@ -34,7 +33,6 @@ public class StreamIterationSink<IN extends Tuple> extends
private static final Log LOG = LogFactory.getLog(StreamIterationSink.class);
- MutableObjectIterator<StreamRecord<IN>> inputIter;
private String iterationId;
@SuppressWarnings("rawtypes")
private BlockingQueue<StreamRecord> dataChannel;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/UID.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/UID.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/UID.java
index 0d1ded2..f644f32 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/UID.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/UID.java
@@ -106,11 +106,15 @@ public class UID implements IOReadableWritable, Serializable {
@Override
public boolean equals(Object obj) {
- try {
- UID other = (UID) obj;
- return Arrays.equals(this.getId(), other.getId());
- } catch (ClassCastException e) {
+ if (obj == null) {
return false;
+ } else {
+ try {
+ UID other = (UID) obj;
+ return Arrays.equals(this.getId(), other.getId());
+ } catch (ClassCastException e) {
+ return false;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
index 44a674d..68eb7d4 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
@@ -43,8 +43,8 @@ public class FieldsPartitioner<T> implements StreamPartitioner<T> {
@Override
public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
int numberOfOutputChannels) {
- returnArray[0] = Math.abs(record.getInstance().getField(keyPosition).hashCode())
- % numberOfOutputChannels;
+ returnArray[0] = Math.abs(record.getInstance().getField(keyPosition).hashCode()
+ % numberOfOutputChannels);
return returnArray;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
index f352399..a9f01bd 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.state;
-import java.io.Serializable;
import java.util.Iterator;
import java.util.List;
@@ -32,8 +31,7 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* compose time based window operator by extending this class by splitting the
* stream into multiple mini batches.
*/
-public class SlidingWindowState<T> implements Serializable {
- private static final long serialVersionUID = -2376149970115888901L;
+public class SlidingWindowState<T> {
private long currentRecordCount;
private int fullRecordCount;
private int slideRecordCount;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java
index 0e5a2a8..57655a7 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java
@@ -19,12 +19,8 @@
package org.apache.flink.streaming.state;
-import org.apache.flink.streaming.state.SlidingWindowState;
-import org.apache.flink.streaming.state.TableState;
-import org.apache.flink.streaming.state.TableStateIterator;
-import org.junit.Test;
-
import org.apache.flink.api.java.tuple.Tuple2;
+import org.junit.Test;
public class InternalStateTest {
@@ -53,11 +49,4 @@ public class InternalStateTest {
System.out.println(tuple.getField(0) + ", " + tuple.getField(1));
}
}
-
- @SuppressWarnings({ "rawtypes", "unused" })
- @Test
- public void WindowStateTest() {
- SlidingWindowState state = new SlidingWindowState(100, 20, 10);
-
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
index 0c8673b..178085e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
@@ -35,7 +35,6 @@ public class SlidingWindowStateTest {
private final static int SLIDE_SIZE = 2;
private static final int UNIT = 1;
- @SuppressWarnings("unchecked")
@Test
public void test() {
SlidingWindowState<Integer> state = new SlidingWindowState<Integer>(SLIDING_BATCH_SIZE,
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
index 6e57739..60a74b3 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
@@ -45,7 +45,7 @@ public class CollaborativeFilteringSource implements
if (line == null) {
break;
}
- if (line != "") {
+ if (!line.equals("")) {
String[] items = line.split("\t");
outRecord.f0 = Integer.valueOf(items[0]);
outRecord.f1 = Integer.valueOf(items[1]);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
index 6b38f2f..01cfa57 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
@@ -42,7 +42,7 @@ public class PageRankSource implements SourceFunction<Tuple3<Integer, Integer, L
if (line == null) {
break;
}
- if (line != "") {
+ if (!line.equals("")) {
String[] link = line.split(":");
outRecord.f0 = Integer.valueOf(link[0]);
outRecord.f1 = Integer.valueOf(link[1]);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6766fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
index 7936efc..0b40869 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
@@ -42,7 +42,7 @@ public class SSSPSource implements SourceFunction<Tuple3<Integer, Integer, Long>
if (line == null) {
break;
}
- if (line != "") {
+ if (!line.equals("")) {
String[] link = line.split(":");
outRecord.f0 = Integer.valueOf(link[0]);
outRecord.f1 = Integer.valueOf(link[1]);
[10/51] [abbrv] git commit: [streaming] Updated operators for better
mutability handling
Posted by se...@apache.org.
[streaming] Updated operators for better mutability handling
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/126a1cb7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/126a1cb7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/126a1cb7
Branch: refs/heads/master
Commit: 126a1cb7f02856feb11e653c74fd0b92880baa8f
Parents: 1edd031
Author: gyfora <gy...@gmail.com>
Authored: Wed Jul 23 12:44:29 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:19:17 2014 +0200
----------------------------------------------------------------------
.../operator/BatchReduceInvokable.java | 79 +++++++++++++++++---
.../api/invokable/operator/co/CoInvokable.java | 18 ++++-
.../invokable/operator/co/CoMapInvokable.java | 13 +++-
.../api/streamcomponent/CoStreamTask.java | 2 +-
4 files changed, 92 insertions(+), 20 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/126a1cb7/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 2d2d890..7684f70 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -20,6 +20,8 @@
package org.apache.flink.streaming.api.invokable.operator;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.flink.api.java.functions.GroupReduceFunction;
import org.apache.flink.api.java.tuple.Tuple;
@@ -47,8 +49,73 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
this.window = true;
}
+ private StreamRecord<IN> loadNextRecord() {
+ try {
+ reuse = recordIterator.next(reuse);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ return reuse;
+ }
+
@Override
public void invoke() throws Exception {
+ if (this.isMutable) {
+ mutableInvoke();
+ } else {
+ immutableInvoke();
+ }
+ }
+
+ private void immutableInvoke() throws Exception {
+ List<IN> tupleBatch = new ArrayList<IN>();
+ boolean batchStart;
+
+ if (window) {
+ long startTime = System.currentTimeMillis();
+ while (loadNextRecord() != null) {
+ batchStart = true;
+ do {
+ if (batchStart) {
+ batchStart = false;
+ } else {
+ reuse = loadNextRecord();
+ if (reuse == null) {
+ break;
+ }
+ }
+ tupleBatch.add(reuse.getTuple());
+ resetReuse();
+ } while (System.currentTimeMillis() - startTime < windowSize);
+ reducer.reduce(tupleBatch.iterator(), collector);
+ tupleBatch.clear();
+ startTime = System.currentTimeMillis();
+ }
+ } else {
+ int counter = 0;
+ while (loadNextRecord() != null) {
+ batchStart = true;
+ do {
+ if (batchStart) {
+ batchStart = false;
+ } else {
+ reuse = loadNextRecord();
+ if (reuse == null) {
+ break;
+ }
+ }
+ counter++;
+ tupleBatch.add(reuse.getTuple());
+ resetReuse();
+ } while (counter < batchSize);
+ reducer.reduce(tupleBatch.iterator(), collector);
+ tupleBatch.clear();
+ counter = 0;
+ }
+ }
+ }
+
+ private void mutableInvoke() throws Exception {
BatchIterator<IN> userIterator;
if (window) {
userIterator = new WindowIterator();
@@ -64,18 +131,6 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
} while (reuse != null);
}
- private StreamRecord<IN> loadNextRecord() {
- if (!isMutable) {
- resetReuse();
- }
- try {
- reuse = recordIterator.next(reuse);
- } catch (IOException e) {
- e.printStackTrace();
- }
- return reuse;
- }
-
public class CounterIterator implements BatchIterator<IN> {
private int counter;
private boolean loadedNext;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/126a1cb7/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
index 733b61e..85086f9 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
@@ -36,19 +36,31 @@ public abstract class CoInvokable<IN1 extends Tuple, IN2 extends Tuple, OUT exte
protected MutableObjectIterator<StreamRecord<IN2>> recordIterator2;
protected StreamRecord<IN1> reuse1;
protected StreamRecord<IN2> reuse2;
+ protected StreamRecordSerializer<IN1> serializer1;
+ protected StreamRecordSerializer<IN2> serializer2;
+ protected boolean isMutable;
public void initialize(Collector<OUT> collector,
MutableObjectIterator<StreamRecord<IN1>> recordIterator1,
StreamRecordSerializer<IN1> serializer1,
MutableObjectIterator<StreamRecord<IN2>> recordIterator2,
- StreamRecordSerializer<IN2> serializer2) {
+ StreamRecordSerializer<IN2> serializer2, boolean isMutable) {
this.collector = collector;
-
+
this.recordIterator1 = recordIterator1;
this.reuse1 = serializer1.createInstance();
-
+
this.recordIterator2 = recordIterator2;
this.reuse2 = serializer2.createInstance();
+
+ this.serializer1 = serializer1;
+ this.serializer2 = serializer2;
+ this.isMutable = isMutable;
+ }
+
+ public void resetReuse() {
+ this.reuse1 = serializer1.createInstance();
+ this.reuse2 = serializer2.createInstance();
}
public abstract void invoke() throws Exception;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/126a1cb7/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
index 4a30425..be5c42f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
@@ -35,19 +35,24 @@ public class CoMapInvokable<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tu
// TODO rework this as UnionRecordReader
@Override
public void invoke() throws Exception {
- boolean noMoreRecordOnInput1;
- boolean noMoreRecordOnInput2;
-
+ boolean noMoreRecordOnInput1 = false;
+ boolean noMoreRecordOnInput2 = false;
+
do {
noMoreRecordOnInput1 = recordIterator1.next(reuse1) == null;
if (!noMoreRecordOnInput1) {
collector.collect(mapper.map1(reuse1.getTuple()));
}
-
+
noMoreRecordOnInput2 = recordIterator2.next(reuse2) == null;
if (!noMoreRecordOnInput2) {
collector.collect(mapper.map2(reuse2.getTuple()));
}
+
+ if (!this.isMutable) {
+ resetReuse();
+ }
} while (!noMoreRecordOnInput1 && !noMoreRecordOnInput2);
}
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/126a1cb7/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
index 204bcfe..60a8152 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -135,7 +135,7 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
CoMapInvokable.class, CoInvokable.class);
userFunction = (CoInvokable<IN1, IN2, OUT>) getInvokable(userFunctionClass);
userFunction.initialize(collector, inputIter1, inTupleSerializer1, inputIter2,
- inTupleSerializer2);
+ inTupleSerializer2, isMutable);
}
protected void setConfigInputs() throws StreamComponentException {
[26/51] [abbrv] git commit: [streaming] Twitter connector prototype
Posted by se...@apache.org.
[streaming] Twitter connector prototype
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/b3cd5fd0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/b3cd5fd0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/b3cd5fd0
Branch: refs/heads/master
Commit: b3cd5fd0262dec5277743d1e3f80548bc104dd8e
Parents: ee7c4a8
Author: Eszes Dávid <es...@gmail.com>
Authored: Fri Aug 1 11:54:05 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:11 2014 +0200
----------------------------------------------------------------------
.../connectors/twitter/TwitterLocal.java | 106 ++++++++
.../connectors/twitter/TwitterSource.java | 243 +++++++++++++++++++
.../connectors/twitter/TwitterStreaming.java | 107 ++++++++
3 files changed, 456 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b3cd5fd0/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
new file mode 100644
index 0000000..138fe05
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
@@ -0,0 +1,106 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.connectors.twitter;
+
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.examples.function.JSONParseFlatMap;
+import org.apache.flink.streaming.examples.wordcount.WordCountCounter;
+import org.apache.flink.util.Collector;
+
+/**
+ * This program demonstrate the use of TwitterSource.
+ * Its aim is to count the frequency of the languages of tweets
+ */
+public class TwitterLocal {
+
+ private static final int PARALLELISM = 1;
+ private static final int SOURCE_PARALLELISM = 1;
+
+ /**
+ * FlatMapFunction to determine the language of tweets if possible
+ */
+ public static class SelectLanguageFlatMap extends
+ JSONParseFlatMap<Tuple1<String>, Tuple1<String>> {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Select the language from the incoming JSON text
+ */
+ @Override
+ public void flatMap(Tuple1<String> value, Collector<Tuple1<String>> out) throws Exception {
+
+ out.collect(new Tuple1<String>(colationOfNull(getField(value.f0, "lang"))));
+ }
+
+ /**
+ * Change the null String to space character. Useful when null is undesirable.
+ * @param in
+ * @return
+ */
+ protected String colationOfNull(String in) {
+ if (in == null) {
+ return " ";
+ }
+ return in;
+ }
+ }
+
+ public static void main(String[] args) {
+
+ String path = new String();
+
+ if (args != null && args.length == 1) {
+ path = args[0];
+ } else {
+ System.err.println("USAGE:\n haho TwitterLocal itt <pathToPropertiesFile>");
+ return;
+ }
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment
+ .createLocalEnvironment(PARALLELISM);
+
+ DataStream<Tuple1<String>> streamSource = env.addSource(new TwitterSource(path, 100),
+ SOURCE_PARALLELISM);
+
+
+ DataStream<Tuple2<String, Integer>> dataStream = streamSource
+ .flatMap(new SelectLanguageFlatMap())
+ .partitionBy(0)
+ .map(new WordCountCounter());
+
+ dataStream.addSink(new SinkFunction<Tuple2<String, Integer>>() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void invoke(Tuple2<String, Integer> tuple) {
+ System.out.println(tuple);
+
+ }
+ });
+
+ env.execute();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b3cd5fd0/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
new file mode 100644
index 0000000..bbff732
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
@@ -0,0 +1,243 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.connectors.twitter;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.util.Collector;
+
+import com.twitter.hbc.ClientBuilder;
+import com.twitter.hbc.core.Constants;
+import com.twitter.hbc.core.endpoint.StatusesSampleEndpoint;
+import com.twitter.hbc.core.processor.StringDelimitedProcessor;
+import com.twitter.hbc.httpclient.BasicClient;
+import com.twitter.hbc.httpclient.auth.Authentication;
+import com.twitter.hbc.httpclient.auth.OAuth1;
+
+/**
+ * Implementation of {@link SourceFunction} specialized to emit tweets from Twitter.
+ * It can connect to Twitter Streaming API, collect tweets and
+ */
+public class TwitterSource extends SourceFunction<Tuple1<String>> {
+
+ private static final Log LOG = LogFactory.getLog(DataStream.class);
+
+ private static final long serialVersionUID = 1L;
+ private String authPath;
+ private transient BlockingQueue<String> queue;
+ private int queueSize = 10000;
+ private transient BasicClient client;
+ private int waitSec = 5;
+
+ private boolean streaming;
+ private int numberOfTweets;
+
+ /**
+ * Create {@link TwitterSource} for streaming
+ * @param authPath
+ * Location of the properties file containing the required authentication information.
+ */
+ public TwitterSource(String authPath) {
+ this.authPath = authPath;
+ streaming = true;
+ }
+
+ /**
+ * Create {@link TwitterSource} to
+ * collect finite number of tweets
+ * @param authPath
+ * Location of the properties file containing the required authentication information.
+ * @param numberOfTweets
+ *
+ */
+ public TwitterSource(String authPath, int numberOfTweets) {
+ this.authPath = authPath;
+ streaming = false;
+ this.numberOfTweets = numberOfTweets;
+ }
+
+ /**
+ *
+ */
+ @Override
+ public void invoke(Collector<Tuple1<String>> collector) throws Exception {
+
+ initializeConnection();
+
+
+ if (streaming) {
+ collectMessages(collector);
+ } else {
+ collectMessages(collector, numberOfTweets);
+ }
+
+ closeConnection();
+ }
+
+ /**
+ * Initialize Hosebird Client to be able to consume Twitter's Streaming API
+ */
+ private void initializeConnection() {
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Initializing Twitter Streaming API connection");
+ }
+
+ queue = new LinkedBlockingQueue<String>(queueSize);
+
+ StatusesSampleEndpoint endpoint = new StatusesSampleEndpoint();
+ endpoint.stallWarnings(false);
+
+ Authentication auth = authenticate();
+
+ initializeClient(endpoint, auth);
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Twitter Streaming API connection established successfully");
+ }
+ }
+
+ private OAuth1 authenticate() {
+
+ Properties authenticationProperties = loadAuthenticationProperties();
+
+ return new OAuth1(authenticationProperties.getProperty("consumerKey"),
+ authenticationProperties.getProperty("consumerSecret"),
+ authenticationProperties.getProperty("token"),
+ authenticationProperties.getProperty("secret"));
+ }
+
+ /**
+ * Reads the given properties file for the authentication data.
+ * @return
+ * the authentication data.
+ */
+ private Properties loadAuthenticationProperties() {
+ Properties properties = new Properties();
+ try {
+ InputStream input = new FileInputStream(authPath);
+ properties.load(input);
+ input.close();
+ } catch (IOException ioe) {
+ new RuntimeException("Cannot open .properties file: " + authPath,
+ ioe);
+ }
+ return properties;
+ }
+
+ private void initializeClient(StatusesSampleEndpoint endpoint,
+ Authentication auth) {
+
+ client = new ClientBuilder().name("twitterSourceClient")
+ .hosts(Constants.STREAM_HOST).endpoint(endpoint)
+ .authentication(auth)
+ .processor(new StringDelimitedProcessor(queue)).build();
+
+ client.connect();
+ }
+
+ /**
+ * Put tweets into collector
+ * @param collector
+ * @param piece
+ */
+ protected void collectMessages(Collector<Tuple1<String>> collector, int piece) {
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Collecting tweets");
+ }
+
+ for (int i = 0; i < piece; i++) {
+ collectOneMessage(collector);
+ }
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Collecting tweets finished");
+ }
+ }
+
+ /**
+ * Put tweets into collector
+ * @param collector
+ *
+ */
+ protected void collectMessages(Collector<Tuple1<String>> collector) {
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Tweet-stream begins");
+ }
+
+ while (true) {
+ collectOneMessage(collector);
+ }
+ }
+
+ /**
+ * Put one tweet into the collector.
+ * @param collector
+ */
+ protected void collectOneMessage(Collector<Tuple1<String>> collector) {
+ if (client.isDone()) {
+ if (LOG.isErrorEnabled()) {
+ LOG.error("Client connection closed unexpectedly: "
+ + client.getExitEvent().getMessage());
+ }
+ }
+
+ try {
+ String msg = queue.poll(waitSec, TimeUnit.SECONDS);
+ if (msg != null) {
+ collector.collect(new Tuple1<String>(msg));
+ } else {
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Did not receive a message in " + waitSec
+ + " seconds");
+ }
+ }
+ } catch (InterruptedException e) {
+ new RuntimeException("'Waiting for tweet' thread is interrupted", e);
+ }
+
+ }
+
+ private void closeConnection() {
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Initiating connection close");
+ }
+
+ client.stop();
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Connection closed successfully");
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b3cd5fd0/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
new file mode 100644
index 0000000..805bf06
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
@@ -0,0 +1,107 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.connectors.twitter;
+
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.examples.function.JSONParseFlatMap;
+import org.apache.flink.util.Collector;
+
+public class TwitterStreaming {
+
+ private static final int PARALLELISM = 1;
+ private static final int SOURCE_PARALLELISM = 1;
+
+ public static class TwitterSink extends SinkFunction<Tuple5<Long, Long, String, String, String>> {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void invoke(Tuple5<Long, Long, String, String, String> tuple) {
+ System.out.println(tuple.f0 + " " + tuple.f1 + " " + tuple.f4);
+ System.out.println("NAME: " + tuple.f2);
+ System.out.println(tuple.f3);
+ System.out.println(" ");
+ }
+
+ }
+
+ public static class SelectDataFlatMap extends
+ JSONParseFlatMap<Tuple1<String>, Tuple5<Long, Long, String, String, String>> {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void flatMap(Tuple1<String> value,
+ Collector<Tuple5<Long, Long, String, String, String>> out)
+ throws Exception {
+
+ out.collect(new Tuple5<Long, Long, String, String, String>(
+ convertDateString2Long(getField(value.f0, "id")),
+ convertDateString2LongDate(getField(value.f0, "created_at")),
+ colationOfNull(getField(value.f0, "user.name")),
+ colationOfNull(getField(value.f0, "text")),
+ getField(value.f0, "lang")));
+ }
+
+ protected String colationOfNull(String in){
+ if(in==null){
+ return " ";
+ }
+ return in;
+ }
+
+ protected Long convertDateString2LongDate(String dateString) {
+ if (dateString!=(null)) {
+ String[] dateArray = dateString.split(" ");
+ return Long.parseLong(dateArray[2])*100000+Long.parseLong(dateArray[5]);
+ }
+ return 0L;
+ }
+
+ protected Long convertDateString2Long(String dateString) {
+ if (dateString != null) {
+ return Long.parseLong(dateString);
+ }
+ return 0L;
+ }
+ }
+
+ public static void main(String[] args) {
+
+ String path = "/home/eszes/git/auth.properties";
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment
+ .createLocalEnvironment(PARALLELISM);
+
+ DataStream<Tuple1<String>> streamSource = env.addSource(
+ new TwitterSource(path,100), SOURCE_PARALLELISM);
+
+ DataStream<Tuple5<Long, Long, String, String, String>> selectedDataStream = streamSource
+ .flatMap(new SelectDataFlatMap());
+
+ selectedDataStream.addSink(new TwitterSink());
+
+ env.execute();
+ }
+}
[09/51] [abbrv] git commit: [streaming] Replaced connection types
with StreamPartitioner in DataStream
Posted by se...@apache.org.
[streaming] Replaced connection types with StreamPartitioner in DataStream
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/1fccb10f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/1fccb10f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/1fccb10f
Branch: refs/heads/master
Commit: 1fccb10ffe79afaaff4a6e810b7572c7a952676a
Parents: 126a1cb
Author: ghermann <re...@gmail.com>
Authored: Thu Jul 24 10:57:44 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:19:17 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 63 +++------
.../flink/streaming/api/JobGraphBuilder.java | 139 -------------------
.../api/StreamExecutionEnvironment.java | 37 ++---
3 files changed, 33 insertions(+), 206 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1fccb10f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index 23f8408..d32aa18 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -27,7 +27,6 @@ import org.apache.flink.api.java.functions.FlatMapFunction;
import org.apache.flink.api.java.functions.GroupReduceFunction;
import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment.ConnectionType;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
@@ -38,6 +37,12 @@ import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
+import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
+import org.apache.flink.streaming.partitioner.DistributePartitioner;
+import org.apache.flink.streaming.partitioner.FieldsPartitioner;
+import org.apache.flink.streaming.partitioner.ForwardPartitioner;
+import org.apache.flink.streaming.partitioner.ShufflePartitioner;
+import org.apache.flink.streaming.partitioner.StreamPartitioner;
/**
* A DataStream represents a stream of elements of the same type. A DataStream
@@ -62,8 +67,7 @@ public class DataStream<T extends Tuple> {
protected String userDefinedName;
protected OutputSelector<T> outputSelector;
protected List<String> connectIDs;
- protected List<ConnectionType> ctypes;
- protected List<Integer> cparams;
+ protected List<StreamPartitioner<T>> partitioners;
protected boolean iterationflag;
protected Integer iterationID;
@@ -103,8 +107,7 @@ public class DataStream<T extends Tuple> {
this.userDefinedName = dataStream.userDefinedName;
this.outputSelector = dataStream.outputSelector;
this.connectIDs = new ArrayList<String>(dataStream.connectIDs);
- this.ctypes = new ArrayList<StreamExecutionEnvironment.ConnectionType>(dataStream.ctypes);
- this.cparams = new ArrayList<Integer>(dataStream.cparams);
+ this.partitioners = new ArrayList<StreamPartitioner<T>>(dataStream.partitioners);
this.iterationflag = dataStream.iterationflag;
this.iterationID = dataStream.iterationID;
}
@@ -116,11 +119,8 @@ public class DataStream<T extends Tuple> {
private void initConnections() {
connectIDs = new ArrayList<String>();
connectIDs.add(getId());
- ctypes = new ArrayList<StreamExecutionEnvironment.ConnectionType>();
- ctypes.add(ConnectionType.SHUFFLE);
- cparams = new ArrayList<Integer>();
- cparams.add(0);
-
+ partitioners = new ArrayList<StreamPartitioner<T>>();
+ partitioners.add(new ShufflePartitioner<T>());
}
/**
@@ -226,8 +226,7 @@ public class DataStream<T extends Tuple> {
*/
private DataStream<T> addConnection(DataStream<T> returnStream, DataStream<T> stream) {
returnStream.connectIDs.addAll(stream.connectIDs);
- returnStream.ctypes.addAll(stream.ctypes);
- returnStream.cparams.addAll(stream.cparams);
+ returnStream.partitioners.addAll(stream.partitioners);
return returnStream;
}
@@ -261,13 +260,7 @@ public class DataStream<T extends Tuple> {
throw new IllegalArgumentException("The position of the field must be non-negative");
}
- DataStream<T> returnStream = new DataStream<T>(this);
-
- for (int i = 0; i < returnStream.ctypes.size(); i++) {
- returnStream.ctypes.set(i, ConnectionType.FIELD);
- returnStream.cparams.set(i, keyposition);
- }
- return returnStream;
+ return setConnectionType(new FieldsPartitioner<T>(keyposition));
}
/**
@@ -277,12 +270,7 @@ public class DataStream<T extends Tuple> {
* @return The DataStream with broadcast partitioning set.
*/
public DataStream<T> broadcast() {
- DataStream<T> returnStream = new DataStream<T>(this);
-
- for (int i = 0; i < returnStream.ctypes.size(); i++) {
- returnStream.ctypes.set(i, ConnectionType.BROADCAST);
- }
- return returnStream;
+ return setConnectionType(new BroadcastPartitioner<T>());
}
/**
@@ -292,12 +280,7 @@ public class DataStream<T extends Tuple> {
* @return The DataStream with shuffle partitioning set.
*/
public DataStream<T> shuffle() {
- DataStream<T> returnStream = new DataStream<T>(this);
-
- for (int i = 0; i < returnStream.ctypes.size(); i++) {
- returnStream.ctypes.set(i, ConnectionType.SHUFFLE);
- }
- return returnStream;
+ return setConnectionType(new ShufflePartitioner<T>());
}
/**
@@ -307,12 +290,7 @@ public class DataStream<T extends Tuple> {
* @return The DataStream with shuffle partitioning set.
*/
public DataStream<T> forward() {
- DataStream<T> returnStream = new DataStream<T>(this);
-
- for (int i = 0; i < returnStream.ctypes.size(); i++) {
- returnStream.ctypes.set(i, ConnectionType.FORWARD);
- }
- return returnStream;
+ return setConnectionType(new ForwardPartitioner<T>());
}
/**
@@ -322,14 +300,19 @@ public class DataStream<T extends Tuple> {
* @return The DataStream with shuffle partitioning set.
*/
public DataStream<T> distribute() {
+ return setConnectionType(new DistributePartitioner<T>());
+ }
+
+ private DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
DataStream<T> returnStream = new DataStream<T>(this);
- for (int i = 0; i < returnStream.ctypes.size(); i++) {
- returnStream.ctypes.set(i, ConnectionType.DISTRIBUTE);
+ for (int i = 0; i < returnStream.partitioners.size(); i++) {
+ returnStream.partitioners.set(i, partitioner);
}
+
return returnStream;
}
-
+
/**
* Applies a Map transformation on a {@link DataStream}. The transformation
* calls a {@link MapFunction} for each element of the DataStream. Each
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1fccb10f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index 986172b..64fdc03 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -53,12 +53,7 @@ import org.apache.flink.streaming.api.streamcomponent.StreamIterationSource;
import org.apache.flink.streaming.api.streamcomponent.StreamSink;
import org.apache.flink.streaming.api.streamcomponent.StreamSource;
import org.apache.flink.streaming.api.streamcomponent.StreamTask;
-import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
-import org.apache.flink.streaming.partitioner.DistributePartitioner;
-import org.apache.flink.streaming.partitioner.FieldsPartitioner;
import org.apache.flink.streaming.partitioner.ForwardPartitioner;
-import org.apache.flink.streaming.partitioner.GlobalPartitioner;
-import org.apache.flink.streaming.partitioner.ShufflePartitioner;
import org.apache.flink.streaming.partitioner.StreamPartitioner;
/**
@@ -463,140 +458,6 @@ public class JobGraphBuilder {
}
/**
- * Connects two components with the given names by broadcast partitioning.
- * <p>
- * Broadcast partitioning: All the emitted tuples are replicated to all of
- * the output instances
- *
- * @param inputStream
- * The DataStream object of the input
- * @param upStreamComponentName
- * Name of the upstream component, that will emit the records
- * @param downStreamComponentName
- * Name of the downstream component, that will receive the
- * records
- * @param typeNumber
- * Number of the type (used at co-functions)
- */
- public <T extends Tuple> void broadcastConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName, int typeNumber) {
- setEdge(upStreamComponentName, downStreamComponentName, new BroadcastPartitioner<T>(),
- typeNumber);
- }
-
- /**
- * Connects two components with the given names by fields partitioning on
- * the given field.
- * <p>
- * Fields partitioning: Tuples are hashed by the given key, and grouped to
- * outputs accordingly
- *
- * @param inputStream
- * The DataStream object of the input
- * @param upStreamComponentName
- * Name of the upstream component, that will emit the records
- * @param downStreamComponentName
- * Name of the downstream component, that will receive the
- * records
- * @param keyPosition
- * Position of key in the tuple
- * @param typeNumber
- * Number of the type (used at co-functions)
- */
- public <T extends Tuple> void fieldsConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName, int keyPosition,
- int typeNumber) {
-
- setEdge(upStreamComponentName, downStreamComponentName, new FieldsPartitioner<T>(
- keyPosition), typeNumber);
- }
-
- /**
- * Connects two components with the given names by global partitioning.
- * <p>
- * Global partitioning: sends all emitted tuples to one output instance
- * (i.e. the first one)
- *
- * @param inputStream
- * The DataStream object of the input
- * @param upStreamComponentName
- * Name of the upstream component, that will emit the tuples
- * @param downStreamComponentName
- * Name of the downstream component, that will receive the tuples
- * @param typeNumber
- * Number of the type (used at co-functions)
- */
- public <T extends Tuple> void globalConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName, int typeNumber) {
- setEdge(upStreamComponentName, downStreamComponentName, new GlobalPartitioner<T>(),
- typeNumber);
- }
-
- /**
- * Connects two components with the given names by shuffle partitioning.
- * <p>
- * Shuffle partitioning: sends the output tuples to a randomly selected
- * channel
- *
- * @param inputStream
- * The DataStream object of the input
- * @param upStreamComponentName
- * Name of the upstream component, that will emit the tuples
- * @param downStreamComponentName
- * Name of the downstream component, that will receive the tuples
- * @param typeNumber
- * Number of the type (used at co-functions)
- */
- public <T extends Tuple> void shuffleConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName, int typeNumber) {
- setEdge(upStreamComponentName, downStreamComponentName, new ShufflePartitioner<T>(),
- typeNumber);
- }
-
- /**
- * Connects two components with the given names by connecting the local
- * subtasks in memory.
- * <p>
- * Forward partitioning: sends the output tuples to the local subtask of the
- * output vertex
- *
- * @param inputStream
- * The DataStream object of the input
- * @param upStreamComponentName
- * Name of the upstream component, that will emit the tuples
- * @param downStreamComponentName
- * Name of the downstream component, that will receive the tuples
- * @param typeNumber
- * Number of the type (used at co-functions)
- */
- public <T extends Tuple> void forwardConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName, int typeNumber) {
- setEdge(upStreamComponentName, downStreamComponentName, new ForwardPartitioner<T>(),
- typeNumber);
- }
-
- /**
- * Connects two components with the given names by distribute partitioning.
- * <p>
- * Distribute partitioning: sends the output tuples evenly distributed along
- * the selected channels
- *
- * @param inputStream
- * The DataStream object of the input
- * @param upStreamComponentName
- * Name of the upstream component, that will emit the tuples
- * @param downStreamComponentName
- * Name of the downstream component, that will receive the tuples
- * @param typeNumber
- * Number of the type (used at co-functions)
- */
- public <T extends Tuple> void distributeConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName, int typeNumber) {
- setEdge(upStreamComponentName, downStreamComponentName, new DistributePartitioner<T>(),
- typeNumber);
- }
-
- /**
* Connects to JobGraph components with the given names, partitioning and
* channel type
*
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1fccb10f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index 4b5bc98..35cfc24 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -43,6 +43,8 @@ import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.api.invokable.SinkInvokable;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
+import org.apache.flink.streaming.partitioner.ForwardPartitioner;
+import org.apache.flink.streaming.partitioner.StreamPartitioner;
/**
* {@link ExecutionEnvironment} for streaming jobs. An instance of it is
@@ -364,9 +366,8 @@ public abstract class StreamExecutionEnvironment {
jobGraphBuilder.setIterationSourceParallelism(iterationID, inputStream.getParallelism());
for (int i = 0; i < inputStream.connectIDs.size(); i++) {
- String input = inputStream.connectIDs.get(i);
- jobGraphBuilder.forwardConnect(inputStream, input, returnStream.getId(), 0);
-
+ String inputID = inputStream.connectIDs.get(i);
+ jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<T>(), 0);
}
}
@@ -552,32 +553,14 @@ public abstract class StreamExecutionEnvironment {
* Number of the type (used at co-functions)
*/
private <T extends Tuple> void connectGraph(DataStream<T> inputStream, String outputID, int typeNumber) {
-
+
for (int i = 0; i < inputStream.connectIDs.size(); i++) {
- ConnectionType type = inputStream.ctypes.get(i);
- String input = inputStream.connectIDs.get(i);
- int param = inputStream.cparams.get(i);
-
- switch (type) {
- case SHUFFLE:
- jobGraphBuilder.shuffleConnect(inputStream, input, outputID, typeNumber);
- break;
- case BROADCAST:
- jobGraphBuilder.broadcastConnect(inputStream, input, outputID, typeNumber);
- break;
- case FIELD:
- jobGraphBuilder.fieldsConnect(inputStream, input, outputID, param, typeNumber);
- break;
- case FORWARD:
- jobGraphBuilder.forwardConnect(inputStream, input, outputID, typeNumber);
- break;
- case DISTRIBUTE:
- jobGraphBuilder.distributeConnect(inputStream, input, outputID, typeNumber);
- break;
- }
-
+ String inputID = inputStream.connectIDs.get(i);
+ StreamPartitioner<T> partitioner = inputStream.partitioners.get(i);
+
+ jobGraphBuilder.setEdge(inputID, outputID, partitioner,
+ typeNumber);
}
-
}
protected <T extends Tuple> void setName(DataStream<T> stream, String name) {
[06/51] [abbrv] git commit: [streaming] Added mutability switch for
operators + cleanup
Posted by se...@apache.org.
[streaming] Added mutability switch for operators + cleanup
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/c21f0e35
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/c21f0e35
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/c21f0e35
Branch: refs/heads/master
Commit: c21f0e351e0976b66436ee55a5bc12fb461ae3fc
Parents: 0c1ef6d
Author: gyfora <gy...@gmail.com>
Authored: Mon Jul 21 11:24:49 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:14:13 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 6 ++
.../flink/streaming/api/JobGraphBuilder.java | 19 ++++--
.../api/StreamExecutionEnvironment.java | 52 ++++++++-------
.../api/invokable/DefaultSinkInvokable.java | 30 ---------
.../api/invokable/DefaultSourceInvokable.java | 35 ----------
.../api/invokable/DefaultTaskInvokable.java | 31 ---------
.../streaming/api/invokable/SinkInvokable.java | 15 +++--
.../api/invokable/StreamRecordInvokable.java | 10 ++-
.../api/invokable/UserSinkInvokable.java | 31 ---------
.../operator/BatchReduceInvokable.java | 17 +++--
.../api/invokable/operator/FilterInvokable.java | 16 ++++-
.../invokable/operator/FlatMapInvokable.java | 11 +++-
.../api/invokable/operator/MapInvokable.java | 11 +++-
.../AbstractStreamComponent.java | 2 +
.../api/streamcomponent/StreamSink.java | 11 ++--
.../api/streamcomponent/StreamSource.java | 3 +-
.../api/streamcomponent/StreamTask.java | 5 +-
.../api/streamrecord/StreamRecordTest.java | 68 --------------------
.../partitioner/BroadcastPartitionerTest.java | 10 +--
.../partitioner/FieldsPartitionerTest.java | 37 ++++++-----
.../partitioner/GlobalPartitionerTest.java | 14 ++--
.../partitioner/ShufflePartitionerTest.java | 24 +++----
22 files changed, 173 insertions(+), 285 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index e3d19f9..d327fad 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -129,6 +129,11 @@ public class DataStream<T extends Tuple> {
public String getId() {
return id;
}
+
+ public DataStream<T> setMutability(boolean isMutable){
+ environment.setMutability(this,isMutable);
+ return this;
+ }
/**
* Sets the degree of parallelism for this operator. The degree must be 1 or
@@ -590,6 +595,7 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsCsv(String path, int batchSize, T endTuple) {
+ setMutability(false);
environment.writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, endTuple);
return new DataStream<T>(this);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index 02a73b1..2b3ee5a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -43,8 +43,8 @@ import org.apache.flink.runtime.jobgraph.JobTaskVertex;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.operators.util.TaskConfig;
import org.apache.flink.streaming.api.collector.OutputSelector;
+import org.apache.flink.streaming.api.invokable.SinkInvokable;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
-import org.apache.flink.streaming.api.invokable.UserSinkInvokable;
import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
import org.apache.flink.streaming.api.streamcomponent.StreamIterationSink;
@@ -70,6 +70,7 @@ public class JobGraphBuilder {
// Graph attributes
private Map<String, AbstractJobVertex> components;
private Map<String, Integer> componentParallelism;
+ private Map<String, Boolean> mutability;
private Map<String, List<String>> outEdgeList;
private Map<String, List<String>> inEdgeList;
private Map<String, List<StreamPartitioner<? extends Tuple>>> connectionTypes;
@@ -100,6 +101,7 @@ public class JobGraphBuilder {
components = new HashMap<String, AbstractJobVertex>();
componentParallelism = new HashMap<String, Integer>();
+ mutability = new HashMap<String, Boolean>();
outEdgeList = new HashMap<String, List<String>>();
inEdgeList = new HashMap<String, List<String>>();
connectionTypes = new HashMap<String, List<StreamPartitioner<? extends Tuple>>>();
@@ -216,7 +218,7 @@ public class JobGraphBuilder {
* @param parallelism
* Number of parallel instances created
*/
- public void addSink(String componentName, UserSinkInvokable<? extends Tuple> InvokableObject,
+ public void addSink(String componentName, SinkInvokable<? extends Tuple> InvokableObject,
String operatorName, byte[] serializedFunction, int parallelism) {
addComponent(componentName, StreamSink.class, InvokableObject, operatorName,
@@ -286,6 +288,7 @@ public class JobGraphBuilder {
componentClasses.put(componentName, componentClass);
setParallelism(componentName, parallelism);
+ mutability.put(componentName, false);
invokableObjects.put(componentName, invokableObject);
operatorNames.put(componentName, operatorName);
serializedFunctions.put(componentName, serializedFunction);
@@ -333,6 +336,8 @@ public class JobGraphBuilder {
Configuration config = new TaskConfig(component.getConfiguration()).getConfiguration();
+ config.setBoolean("isMutable", mutability.get(componentName));
+
// Set vertex config
if (invokableObject != null) {
config.setClass("userfunction", invokableObject.getClass());
@@ -423,6 +428,10 @@ public class JobGraphBuilder {
componentParallelism.put(componentName, parallelism);
}
+ public void setMutability(String componentName, boolean isMutable) {
+ mutability.put(componentName, isMutable);
+ }
+
/**
* Connects two vertices in the JobGraph using the selected partitioner
* settings
@@ -574,8 +583,8 @@ public class JobGraphBuilder {
}
} catch (JobGraphDefinitionException e) {
- throw new RuntimeException("Cannot connect components: "
- + upStreamComponentName + " to " + downStreamComponentName, e);
+ throw new RuntimeException("Cannot connect components: " + upStreamComponentName
+ + " to " + downStreamComponentName, e);
}
int outputIndex = upStreamComponent.getNumberOfForwardConnections() - 1;
@@ -638,7 +647,7 @@ public class JobGraphBuilder {
public <T extends Tuple> void setOutputSelector(String componentName,
byte[] serializedOutputSelector) {
outputSelectors.put(componentName, serializedOutputSelector);
-
+
if (LOG.isDebugEnabled()) {
LOG.debug("Outputselector set for " + componentName);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index a14b150..c1eca4a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -121,6 +121,10 @@ public abstract class StreamExecutionEnvironment {
this.degreeOfParallelism = degreeOfParallelism;
}
+ protected void setMutability(DataStream<?> stream, boolean isMutable) {
+ jobGraphBuilder.setMutability(stream.getId(), isMutable);
+ }
+
/**
* Sets the number of hardware contexts (CPU cores / threads) used when
* executed in {@link LocalStreamEnvironment}.
@@ -226,7 +230,7 @@ public abstract class StreamExecutionEnvironment {
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize collection");
}
-
+
return returnStream;
}
@@ -331,7 +335,7 @@ public abstract class StreamExecutionEnvironment {
jobGraphBuilder.addIterationSink(returnStream.getId(), inputStream.getId(), iterationID,
inputStream.getParallelism(), iterationName);
-
+
jobGraphBuilder.setIterationSourceParallelism(iterationID, inputStream.getParallelism());
for (int i = 0; i < inputStream.connectIDs.size(); i++) {
@@ -396,7 +400,7 @@ public abstract class StreamExecutionEnvironment {
return returnStream;
}
-
+
/**
* Writes a DataStream to the file specified by path in text format. The
* writing is performed periodically, in every millis milliseconds. For
@@ -420,6 +424,7 @@ public abstract class StreamExecutionEnvironment {
DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<T>(path,
format, millis, endTuple));
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
return returnStream;
}
@@ -447,6 +452,7 @@ public abstract class StreamExecutionEnvironment {
DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<T>(path,
format, batchSize, endTuple));
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
return returnStream;
}
@@ -473,6 +479,7 @@ public abstract class StreamExecutionEnvironment {
DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<T>(path,
format, millis, endTuple));
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
return returnStream;
}
@@ -500,6 +507,7 @@ public abstract class StreamExecutionEnvironment {
DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<T>(path,
format, batchSize, endTuple));
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
return returnStream;
}
@@ -558,25 +566,25 @@ public abstract class StreamExecutionEnvironment {
jobGraphBuilder.setParallelism(inputStream.getId(), inputStream.degreeOfParallelism);
}
-// /**
-// * Converts object to byte array using default java serialization
-// *
-// * @param object
-// * Object to be serialized
-// * @return Serialized object
-// */
-// static byte[] serializeToByteArray(Serializable object) {
-// SerializationUtils.serialize(object);
-// ByteArrayOutputStream baos = new ByteArrayOutputStream();
-// ObjectOutputStream oos;
-// try {
-// oos = new ObjectOutputStream(baos);
-// oos.writeObject(object);
-// } catch (IOException e) {
-// throw new RuntimeException("Cannot serialize object: " + object);
-// }
-// return baos.toByteArray();
-// }
+ // /**
+ // * Converts object to byte array using default java serialization
+ // *
+ // * @param object
+ // * Object to be serialized
+ // * @return Serialized object
+ // */
+ // static byte[] serializeToByteArray(Serializable object) {
+ // SerializationUtils.serialize(object);
+ // ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ // ObjectOutputStream oos;
+ // try {
+ // oos = new ObjectOutputStream(baos);
+ // oos.writeObject(object);
+ // } catch (IOException e) {
+ // throw new RuntimeException("Cannot serialize object: " + object);
+ // }
+ // return baos.toByteArray();
+ // }
// --------------------------------------------------------------------------------------------
// Instantiation of Execution Contexts
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSinkInvokable.java
deleted file mode 100644
index 09e766b..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSinkInvokable.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.invokable;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-public class DefaultSinkInvokable<T extends Tuple> extends UserSinkInvokable<T> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke() throws Exception {
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSourceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSourceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSourceInvokable.java
deleted file mode 100644
index 0920ea0..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSourceInvokable.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.invokable;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.util.Collector;
-
-public class DefaultSourceInvokable extends UserSourceInvokable<Tuple> {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Collector<Tuple> collector) throws Exception {
- collector.collect(new Tuple1<String>("hello flink!"));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultTaskInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultTaskInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultTaskInvokable.java
deleted file mode 100644
index c86ffcb..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultTaskInvokable.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.invokable;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-public class DefaultTaskInvokable<T extends Tuple> extends UserTaskInvokable<T, T> {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke() throws Exception {
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index c6e4e36..3c14490 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -22,7 +22,7 @@ package org.apache.flink.streaming.api.invokable;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
-public class SinkInvokable<IN extends Tuple> extends UserSinkInvokable<IN> {
+public class SinkInvokable<IN extends Tuple> extends StreamRecordInvokable<IN, IN> {
private static final long serialVersionUID = 1L;
private SinkFunction<IN> sinkFunction;
@@ -33,8 +33,15 @@ public class SinkInvokable<IN extends Tuple> extends UserSinkInvokable<IN> {
@Override
public void invoke() throws Exception {
- while ((reuse = recordIterator.next(reuse)) != null) {
- sinkFunction.invoke((IN) reuse.getTuple());
- }
+ if (this.isMutable) {
+ while (recordIterator.next(reuse) != null) {
+ sinkFunction.invoke((IN) reuse.getTuple());
+ }
+ } else {
+ while (recordIterator.next(reuse) != null) {
+ sinkFunction.invoke((IN) reuse.getTuple());
+ resetReuse();
+ }
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
index 98eb679..903372b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
@@ -32,13 +32,21 @@ public abstract class StreamRecordInvokable<IN extends Tuple, OUT extends Tuple>
protected Collector<OUT> collector;
protected MutableObjectIterator<StreamRecord<IN>> recordIterator;
+ StreamRecordSerializer<IN> serializer;
protected StreamRecord<IN> reuse;
+ protected boolean isMutable;
public void initialize(Collector<OUT> collector,
MutableObjectIterator<StreamRecord<IN>> recordIterator,
- StreamRecordSerializer<IN> serializer) {
+ StreamRecordSerializer<IN> serializer, boolean isMutable) {
this.collector = collector;
this.recordIterator = recordIterator;
+ this.serializer = serializer;
+ this.reuse = serializer.createInstance();
+ this.isMutable = isMutable;
+ }
+
+ protected void resetReuse() {
this.reuse = serializer.createInstance();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserSinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserSinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserSinkInvokable.java
deleted file mode 100644
index 78214e6..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserSinkInvokable.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.invokable;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-public abstract class UserSinkInvokable<IN extends Tuple> extends StreamRecordInvokable<IN, IN>
- implements Serializable {
-
-
- private static final long serialVersionUID = 1L;
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 8e35af2..811a929 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -43,10 +43,18 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
@Override
public void invoke() throws Exception {
MyIterator it = new MyIterator();
- do {
- reducer.reduce(it, collector);
- it.reset();
- } while (reuse != null);
+ if (this.isMutable) {
+ do {
+ reducer.reduce(it, collector);
+ it.reset();
+ } while (reuse != null);
+ } else {
+ do {
+ reducer.reduce(it, collector);
+ it.reset();
+ } while (reuse != null);
+ }
+
}
public class MyIterator implements Iterator<IN> {
@@ -62,6 +70,7 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
return false;
} else {
try {
+ resetReuse();
reuse = recordIterator.next(reuse);
} catch (IOException e) {
e.printStackTrace();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
index 1dba6b8..ac79764 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
@@ -35,10 +35,20 @@ public class FilterInvokable<IN extends Tuple> extends UserTaskInvokable<IN, IN>
@Override
public void invoke() throws Exception {
- while ((reuse = recordIterator.next(reuse)) != null) {
- if (filterFunction.filter(reuse.getTuple())) {
- collector.collect(reuse.getTuple());
+ if (this.isMutable) {
+ while (recordIterator.next(reuse) != null) {
+ if (filterFunction.filter(reuse.getTuple())) {
+ collector.collect(reuse.getTuple());
+ }
+ }
+ } else {
+ while (recordIterator.next(reuse) != null) {
+ if (filterFunction.filter(reuse.getTuple())) {
+ collector.collect(reuse.getTuple());
+ }
+ resetReuse();
}
}
+
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
index 36ab1bf..33bda80 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
@@ -34,8 +34,15 @@ public class FlatMapInvokable<IN extends Tuple, OUT extends Tuple> extends
}
public void invoke() throws Exception {
- while ((reuse = recordIterator.next(reuse)) != null) {
- flatMapper.flatMap(reuse.getTuple(), collector);
+ if (this.isMutable) {
+ while (recordIterator.next(reuse) != null) {
+ flatMapper.flatMap(reuse.getTuple(), collector);
+ }
+ } else {
+ while (recordIterator.next(reuse) != null) {
+ flatMapper.flatMap(reuse.getTuple(), collector);
+ resetReuse();
+ }
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
index 59b9f8f..ff29d15 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
@@ -34,8 +34,15 @@ public class MapInvokable<IN extends Tuple, OUT extends Tuple> extends UserTaskI
@Override
public void invoke() throws Exception {
- while ((reuse = recordIterator.next(reuse)) != null) {
- collector.collect(mapper.map(reuse.getTuple()));
+ if (this.isMutable) {
+ while (recordIterator.next(reuse) != null) {
+ collector.collect(mapper.map(reuse.getTuple()));
+ }
+ } else {
+ while (recordIterator.next(reuse) != null) {
+ collector.collect(mapper.map(reuse.getTuple()));
+ resetReuse();
+ }
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index 207d269..f90caf8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -76,6 +76,7 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
protected int instanceID;
protected String name;
private static int numComponents = 0;
+ protected boolean isMutable;
protected static int newComponent() {
numComponents++;
@@ -259,6 +260,7 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
StreamComponentInvokable userFunction = null;
byte[] userFunctionSerialized = configuration.getBytes("serializedudf", null);
+ this.isMutable = configuration.getBoolean("isMutable", false);
try {
userFunction = deserializeObject(userFunctionSerialized);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
index 508a10e9..1433ebc 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
@@ -23,9 +23,8 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.io.network.api.MutableReader;
-import org.apache.flink.streaming.api.invokable.DefaultSinkInvokable;
+import org.apache.flink.streaming.api.invokable.SinkInvokable;
import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
-import org.apache.flink.streaming.api.invokable.UserSinkInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.util.MutableObjectIterator;
@@ -62,10 +61,10 @@ public class StreamSink<IN extends Tuple> extends AbstractStreamComponent<IN, IN
@SuppressWarnings({ "rawtypes", "unchecked" })
@Override
protected void setInvokable() {
- Class<? extends UserSinkInvokable> userFunctionClass = configuration.getClass(
- "userfunction", DefaultSinkInvokable.class, UserSinkInvokable.class);
- userFunction = (UserSinkInvokable<IN>) getInvokable(userFunctionClass);
- userFunction.initialize(collector, inputIter, inTupleSerializer);
+ Class<? extends SinkInvokable> userFunctionClass = configuration.getClass("userfunction",
+ SinkInvokable.class, SinkInvokable.class);
+ userFunction = (SinkInvokable<IN>) getInvokable(userFunctionClass);
+ userFunction.initialize(collector, inputIter, inTupleSerializer, isMutable);
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
index 4ac7100..108916a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
@@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.invokable.DefaultSourceInvokable;
import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -74,7 +73,7 @@ public class StreamSource<OUT extends Tuple> extends AbstractStreamComponent<Tup
protected void setInvokable() {
// Default value is a TaskInvokable even if it was called from a source
Class<? extends UserSourceInvokable> userFunctionClass = configuration.getClass(
- "userfunction", DefaultSourceInvokable.class, UserSourceInvokable.class);
+ "userfunction", UserSourceInvokable.class, UserSourceInvokable.class);
userFunction = (UserSourceInvokable<OUT>) getInvokable(userFunctionClass);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
index 44f93b3..7da19aa 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
@@ -29,7 +29,6 @@ import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.invokable.DefaultTaskInvokable;
import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -79,9 +78,9 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
protected void setInvokable() {
// Default value is a TaskInvokable even if it was called from a source
Class<? extends UserTaskInvokable> userFunctionClass = configuration.getClass(
- "userfunction", DefaultTaskInvokable.class, UserTaskInvokable.class);
+ "userfunction", UserTaskInvokable.class, UserTaskInvokable.class);
userFunction = (UserTaskInvokable<IN, OUT>) getInvokable(userFunctionClass);
- userFunction.initialize(collector, inputIter, inTupleSerializer);
+ userFunction.initialize(collector, inputIter, inTupleSerializer, isMutable);
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamrecord/StreamRecordTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamrecord/StreamRecordTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamrecord/StreamRecordTest.java
deleted file mode 100755
index 403c863..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamrecord/StreamRecordTest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.streamrecord;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-import org.apache.flink.runtime.io.network.serialization.DataInputDeserializer;
-import org.apache.flink.runtime.io.network.serialization.DataOutputSerializer;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.junit.Test;
-
-public class StreamRecordTest {
-
- @Test
- public void testReadWrite() throws IOException {
- StreamRecord<Tuple2<Integer, String>> streamRecord = new StreamRecord<Tuple2<Integer, String>>();
- Tuple2<Integer, String> tuple = new Tuple2<Integer, String>(2, "a");
- streamRecord.setTuple(tuple).setId(1);
-
- TupleSerializer<Tuple2<Integer, String>> ts = (TupleSerializer<Tuple2<Integer, String>>) TypeExtractor
- .getForObject(tuple).createSerializer();
-
- SerializationDelegate<Tuple2<Integer, String>> sd = new SerializationDelegate<Tuple2<Integer, String>>(
- ts);
- streamRecord.setSeralizationDelegate(sd);
-
- DataOutputSerializer out = new DataOutputSerializer(64);
- streamRecord.write(out);
-
- ByteBuffer buff = out.wrapAsByteBuffer();
-
- DataInputDeserializer in = new DataInputDeserializer(buff);
-
- StreamRecord<Tuple2<Integer, String>> streamRecord2 = new StreamRecord<Tuple2<Integer, String>>();
-
- streamRecord2.setDeseralizationDelegate(
- new DeserializationDelegate<Tuple2<Integer, String>>(ts), ts);
-
- streamRecord2.read(in);
-
- assertEquals(streamRecord.getTuple(), streamRecord2.getTuple());
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/BroadcastPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/BroadcastPartitionerTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/BroadcastPartitionerTest.java
index 3a1c8f0..ac5240f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/BroadcastPartitionerTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/BroadcastPartitionerTest.java
@@ -22,6 +22,7 @@ package org.apache.flink.streaming.partitioner;
import static org.junit.Assert.assertArrayEquals;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.junit.Before;
import org.junit.Test;
@@ -33,6 +34,7 @@ public class BroadcastPartitionerTest {
private BroadcastPartitioner<Tuple> broadcastPartitioner3;
private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>();
+ private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(null);
@Before
public void setPartitioner() {
@@ -47,9 +49,9 @@ public class BroadcastPartitionerTest {
int[] first = new int[] { 0 };
int[] second = new int[] { 0, 1 };
int[] sixth = new int[] { 0, 1, 2, 3, 4, 5 };
-
- assertArrayEquals(first, broadcastPartitioner1.selectChannels(streamRecord, 1));
- assertArrayEquals(second, broadcastPartitioner2.selectChannels(streamRecord, 2));
- assertArrayEquals(sixth, broadcastPartitioner3.selectChannels(streamRecord, 6));
+ sd.setInstance(streamRecord);
+ assertArrayEquals(first, broadcastPartitioner1.selectChannels(sd, 1));
+ assertArrayEquals(second, broadcastPartitioner2.selectChannels(sd, 2));
+ assertArrayEquals(sixth, broadcastPartitioner3.selectChannels(sd, 6));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java
index 77dacf7..716a869 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.junit.Before;
import org.junit.Test;
@@ -31,8 +32,14 @@ import org.junit.Test;
public class FieldsPartitionerTest {
private FieldsPartitioner<Tuple> fieldsPartitioner;
- private StreamRecord<Tuple> streamRecord1 = new StreamRecord<Tuple>().setTuple(new Tuple2<String, Integer>("test", 0));
- private StreamRecord<Tuple> streamRecord2 = new StreamRecord<Tuple>().setTuple(new Tuple2<String, Integer>("test", 42));
+ private StreamRecord<Tuple> streamRecord1 = new StreamRecord<Tuple>()
+ .setTuple(new Tuple2<String, Integer>("test", 0));
+ private StreamRecord<Tuple> streamRecord2 = new StreamRecord<Tuple>()
+ .setTuple(new Tuple2<String, Integer>("test", 42));
+ private SerializationDelegate<StreamRecord<Tuple>> sd1 = new SerializationDelegate<StreamRecord<Tuple>>(
+ null);
+ private SerializationDelegate<StreamRecord<Tuple>> sd2 = new SerializationDelegate<StreamRecord<Tuple>>(
+ null);
@Before
public void setPartitioner() {
@@ -41,22 +48,22 @@ public class FieldsPartitionerTest {
@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);
+ sd1.setInstance(streamRecord1);
+ assertEquals(1, fieldsPartitioner.selectChannels(sd1, 1).length);
+ assertEquals(1, fieldsPartitioner.selectChannels(sd1, 2).length);
+ assertEquals(1, fieldsPartitioner.selectChannels(sd1, 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));
+ sd1.setInstance(streamRecord1);
+ sd2.setInstance(streamRecord2);
+
+ assertArrayEquals(fieldsPartitioner.selectChannels(sd1, 1),
+ fieldsPartitioner.selectChannels(sd2, 1));
+ assertArrayEquals(fieldsPartitioner.selectChannels(sd1, 2),
+ fieldsPartitioner.selectChannels(sd2, 2));
+ assertArrayEquals(fieldsPartitioner.selectChannels(sd1, 1024),
+ fieldsPartitioner.selectChannels(sd2, 1024));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/GlobalPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/GlobalPartitionerTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/GlobalPartitionerTest.java
index dcf3160..e2703c1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/GlobalPartitionerTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/GlobalPartitionerTest.java
@@ -22,6 +22,7 @@ package org.apache.flink.streaming.partitioner;
import static org.junit.Assert.assertArrayEquals;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.junit.Before;
import org.junit.Test;
@@ -30,6 +31,8 @@ public class GlobalPartitionerTest {
private GlobalPartitioner<Tuple> globalPartitioner;
private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>();
+ private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
+ null);
@Before
public void setPartitioner() {
@@ -40,11 +43,10 @@ public class GlobalPartitionerTest {
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));
+ sd.setInstance(streamRecord);
+
+ assertArrayEquals(result, globalPartitioner.selectChannels(sd, 1));
+ assertArrayEquals(result, globalPartitioner.selectChannels(sd, 2));
+ assertArrayEquals(result, globalPartitioner.selectChannels(sd, 1024));
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c21f0e35/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/ShufflePartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/ShufflePartitionerTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/ShufflePartitionerTest.java
index 32cc2ab..33881e8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/ShufflePartitionerTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/ShufflePartitionerTest.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.junit.Before;
import org.junit.Test;
@@ -31,6 +32,8 @@ public class ShufflePartitionerTest {
private ShufflePartitioner<Tuple> shufflePartitioner;
private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>();
+ private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
+ null);
@Before
public void setPartitioner() {
@@ -39,22 +42,21 @@ public class ShufflePartitionerTest {
@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);
+ sd.setInstance(streamRecord);
+ assertEquals(1, shufflePartitioner.selectChannels(sd, 1).length);
+ assertEquals(1, shufflePartitioner.selectChannels(sd, 2).length);
+ assertEquals(1, shufflePartitioner.selectChannels(sd, 1024).length);
}
@Test
public void testSelectChannelsInterval() {
- assertEquals(0, shufflePartitioner.selectChannels(streamRecord, 1)[0]);
+ sd.setInstance(streamRecord);
+ assertEquals(0, shufflePartitioner.selectChannels(sd, 1)[0]);
- assertTrue(0 <= shufflePartitioner.selectChannels(streamRecord, 2)[0]);
- assertTrue(2 > shufflePartitioner.selectChannels(streamRecord, 2)[0]);
+ assertTrue(0 <= shufflePartitioner.selectChannels(sd, 2)[0]);
+ assertTrue(2 > shufflePartitioner.selectChannels(sd, 2)[0]);
- assertTrue(0 <= shufflePartitioner.selectChannels(streamRecord, 1024)[0]);
- assertTrue(1024 > shufflePartitioner.selectChannels(streamRecord, 1024)[0]);
+ assertTrue(0 <= shufflePartitioner.selectChannels(sd, 1024)[0]);
+ assertTrue(1024 > shufflePartitioner.selectChannels(sd, 1024)[0]);
}
}
[16/51] [abbrv] git commit: [streaming] StreamRecordWriter added for
automatic output flushing settings
Posted by se...@apache.org.
[streaming] StreamRecordWriter added for automatic output flushing settings
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/a2c4137f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/a2c4137f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/a2c4137f
Branch: refs/heads/master
Commit: a2c4137feb4c8ad6e0f0b34c4b0062b577af0150
Parents: be459ae
Author: gyfora <gy...@gmail.com>
Authored: Thu Jul 24 11:10:56 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:20:17 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 13 +
.../flink/streaming/api/JobGraphBuilder.java | 321 +++++++++----------
.../api/StreamExecutionEnvironment.java | 4 +
.../AbstractStreamComponent.java | 41 ++-
.../api/streamcomponent/StreamRecordWriter.java | 115 +++++++
.../apache/flink/streaming/api/PrintTest.java | 47 +--
6 files changed, 303 insertions(+), 238 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a2c4137f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index 27e4d89..d965bf2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -149,6 +149,19 @@ public class DataStream<T extends Tuple> {
}
/**
+ * Sets the maximum time frequency (ms) for the flushing of the output
+ * buffer. By default the output buffers flush only when they are full.
+ *
+ * @param timeoutMillis
+ * The maximum time between two output flushes.
+ * @return The DataStream with buffer timeout set.
+ */
+ public DataStream<T> setBufferTimeout(long timeoutMillis) {
+ environment.setBufferTimeout(this, timeoutMillis);
+ return this;
+ }
+
+ /**
* Sets the degree of parallelism for this operator. The degree must be 1 or
* more.
*
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a2c4137f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index 64fdc03..7a10246 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -19,19 +19,13 @@
package org.apache.flink.streaming.api;
-import java.io.ByteArrayOutputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import org.apache.commons.lang3.SerializationUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.network.channels.ChannelType;
import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
import org.apache.flink.runtime.jobgraph.DistributionPattern;
@@ -42,9 +36,9 @@ import org.apache.flink.runtime.jobgraph.JobOutputVertex;
import org.apache.flink.runtime.jobgraph.JobTaskVertex;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.streaming.api.collector.OutputSelector;
+import org.apache.flink.streaming.api.invokable.SinkInvokable;
+import org.apache.flink.streaming.api.invokable.SourceInvokable;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
-import org.apache.flink.streaming.api.invokable.UserSinkInvokable;
-import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
import org.apache.flink.streaming.api.streamcomponent.CoStreamTask;
@@ -55,9 +49,10 @@ import org.apache.flink.streaming.api.streamcomponent.StreamSource;
import org.apache.flink.streaming.api.streamcomponent.StreamTask;
import org.apache.flink.streaming.partitioner.ForwardPartitioner;
import org.apache.flink.streaming.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
/**
- * Object for building Flink stream processing job graphs
+ * Object for building Apache Flink stream processing job graphs
*/
public class JobGraphBuilder {
@@ -67,19 +62,27 @@ public class JobGraphBuilder {
// Graph attributes
private Map<String, AbstractJobVertex> components;
private Map<String, Integer> componentParallelism;
- private Map<String, ArrayList<String>> outEdgeList;
- private Map<String, ArrayList<Integer>> outEdgeType;
+ private Map<String, Long> bufferTimeout;
+ private Map<String, List<String>> outEdgeList;
+ private Map<String, List<Integer>> outEdgeType;
+ private Map<String, List<List<String>>> outEdgeNames;
+ private Map<String, Boolean> mutability;
private Map<String, List<String>> inEdgeList;
- private Map<String, List<StreamPartitioner<? extends Tuple>>> connectionTypes;
- private Map<String, String> userDefinedNames;
+ private Map<String, List<StreamPartitioner<?>>> connectionTypes;
private Map<String, String> operatorNames;
- private Map<String, StreamComponentInvokable> invokableObjects;
+ private Map<String, StreamComponentInvokable<?>> invokableObjects;
+ private Map<String, TypeSerializerWrapper<?, ?, ?>> typeWrappers;
private Map<String, byte[]> serializedFunctions;
private Map<String, byte[]> outputSelectors;
private Map<String, Class<? extends AbstractInvokable>> componentClasses;
private Map<String, String> iterationIds;
- private Map<String, String> iterationHeadNames;
+ private Map<String, String> iterationIDtoSourceName;
+ private Map<String, String> iterationIDtoSinkName;
private Map<String, Integer> iterationTailCount;
+ private Map<String, Long> iterationWaitTime;
+
+ private int degreeOfParallelism;
+ private int executionParallelism;
private String maxParallelismVertexName;
private int maxParallelism;
@@ -98,19 +101,24 @@ public class JobGraphBuilder {
components = new HashMap<String, AbstractJobVertex>();
componentParallelism = new HashMap<String, Integer>();
- outEdgeList = new HashMap<String, ArrayList<String>>();
- outEdgeType = new HashMap<String, ArrayList<Integer>>();
+ bufferTimeout = new HashMap<String, Long>();
+ outEdgeList = new HashMap<String, List<String>>();
+ outEdgeType = new HashMap<String, List<Integer>>();
+ outEdgeNames = new HashMap<String, List<List<String>>>();
+ mutability = new HashMap<String, Boolean>();
inEdgeList = new HashMap<String, List<String>>();
- connectionTypes = new HashMap<String, List<StreamPartitioner<? extends Tuple>>>();
- userDefinedNames = new HashMap<String, String>();
+ connectionTypes = new HashMap<String, List<StreamPartitioner<?>>>();
operatorNames = new HashMap<String, String>();
- invokableObjects = new HashMap<String, StreamComponentInvokable>();
+ invokableObjects = new HashMap<String, StreamComponentInvokable<?>>();
+ typeWrappers = new HashMap<String, TypeSerializerWrapper<?, ?, ?>>();
serializedFunctions = new HashMap<String, byte[]>();
outputSelectors = new HashMap<String, byte[]>();
componentClasses = new HashMap<String, Class<? extends AbstractInvokable>>();
iterationIds = new HashMap<String, String>();
- iterationHeadNames = new HashMap<String, String>();
+ iterationIDtoSourceName = new HashMap<String, String>();
+ iterationIDtoSinkName = new HashMap<String, String>();
iterationTailCount = new HashMap<String, Integer>();
+ iterationWaitTime = new HashMap<String, Long>();
maxParallelismVertexName = "";
maxParallelism = 0;
@@ -119,6 +127,22 @@ public class JobGraphBuilder {
}
}
+ public int getDefaultParallelism() {
+ return degreeOfParallelism;
+ }
+
+ public void setDefaultParallelism(int defaultParallelism) {
+ this.degreeOfParallelism = defaultParallelism;
+ }
+
+ public int getExecutionParallelism() {
+ return executionParallelism;
+ }
+
+ public void setExecutionParallelism(int executionParallelism) {
+ this.executionParallelism = executionParallelism;
+ }
+
/**
* Adds source to the JobGraph with the given parameters
*
@@ -133,11 +157,11 @@ public class JobGraphBuilder {
* @param parallelism
* Number of parallel instances created
*/
- public void addSource(String componentName,
- UserSourceInvokable<? extends Tuple> InvokableObject, String operatorName,
+ public void addSource(String componentName, SourceInvokable<?> InvokableObject,
+ TypeSerializerWrapper<?, ?, ?> typeWrapper, String operatorName,
byte[] serializedFunction, int parallelism) {
- addComponent(componentName, StreamSource.class, InvokableObject, operatorName,
+ addComponent(componentName, StreamSource.class, typeWrapper, InvokableObject, operatorName,
serializedFunction, parallelism);
if (LOG.isDebugEnabled()) {
@@ -157,18 +181,24 @@ public class JobGraphBuilder {
* ID of iteration for multiple iterations
* @param parallelism
* Number of parallel instances created
+ * @param waitTime
+ * Max wait time for next record
*/
public void addIterationSource(String componentName, String iterationHead, String iterationID,
- int parallelism) {
+ int parallelism, long waitTime) {
- addComponent(componentName, StreamIterationSource.class, null, null, null, parallelism);
+ addComponent(componentName, StreamIterationSource.class, null, null, null, null,
+ parallelism);
iterationIds.put(componentName, iterationID);
- iterationHeadNames.put(iterationID, componentName);
+ iterationIDtoSourceName.put(iterationID, componentName);
setBytesFrom(iterationHead, componentName);
setEdge(componentName, iterationHead,
- connectionTypes.get(inEdgeList.get(iterationHead).get(0)).get(0), 0);
+ connectionTypes.get(inEdgeList.get(iterationHead).get(0)).get(0), 0,
+ new ArrayList<String>());
+
+ iterationWaitTime.put(iterationIDtoSourceName.get(iterationID), waitTime);
if (LOG.isDebugEnabled()) {
LOG.debug("ITERATION SOURCE: " + componentName);
@@ -178,8 +208,8 @@ public class JobGraphBuilder {
/**
* Adds a task to the JobGraph with the given parameters
*
- * @param componentName
- * Name of the component
+ * @param componentNameTypeSerializerWrapper
+ * <?, ?, ?> typeWrapper, Name of the component
* @param taskInvokableObject
* User defined operator
* @param operatorName
@@ -189,24 +219,26 @@ public class JobGraphBuilder {
* @param parallelism
* Number of parallel instances created
*/
- public <IN extends Tuple, OUT extends Tuple> void addTask(String componentName,
- UserTaskInvokable<IN, OUT> taskInvokableObject, String operatorName,
+ public <IN, OUT> void addTask(String componentName,
+ UserTaskInvokable<IN, OUT> taskInvokableObject,
+ TypeSerializerWrapper<?, ?, ?> typeWrapper, String operatorName,
byte[] serializedFunction, int parallelism) {
- addComponent(componentName, StreamTask.class, taskInvokableObject, operatorName,
- serializedFunction, parallelism);
+ addComponent(componentName, StreamTask.class, typeWrapper, taskInvokableObject,
+ operatorName, serializedFunction, parallelism);
if (LOG.isDebugEnabled()) {
LOG.debug("TASK: " + componentName);
}
}
- public <IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> void addCoTask(
- String componentName, CoInvokable<IN1, IN2, OUT> taskInvokableObject,
- String operatorName, byte[] serializedFunction, int parallelism) {
+ public <IN1, IN2, OUT> void addCoTask(String componentName,
+ CoInvokable<IN1, IN2, OUT> taskInvokableObject,
+ TypeSerializerWrapper<?, ?, ?> typeWrapper, String operatorName,
+ byte[] serializedFunction, int parallelism) {
- addComponent(componentName, CoStreamTask.class, taskInvokableObject, operatorName,
- serializedFunction, parallelism);
+ addComponent(componentName, CoStreamTask.class, typeWrapper, taskInvokableObject,
+ operatorName, serializedFunction, parallelism);
if (LOG.isDebugEnabled()) {
LOG.debug("CO-TASK: " + componentName);
@@ -227,10 +259,11 @@ public class JobGraphBuilder {
* @param parallelism
* Number of parallel instances created
*/
- public void addSink(String componentName, UserSinkInvokable<? extends Tuple> InvokableObject,
- String operatorName, byte[] serializedFunction, int parallelism) {
+ public void addSink(String componentName, SinkInvokable<?> InvokableObject,
+ TypeSerializerWrapper<?, ?, ?> typeWrapper, String operatorName,
+ byte[] serializedFunction, int parallelism) {
- addComponent(componentName, StreamSink.class, InvokableObject, operatorName,
+ addComponent(componentName, StreamSink.class, typeWrapper, InvokableObject, operatorName,
serializedFunction, parallelism);
if (LOG.isDebugEnabled()) {
@@ -254,19 +287,17 @@ public class JobGraphBuilder {
* Number of parallel instances created
* @param directName
* Id of the output direction
+ * @param waitTime
+ * Max waiting time for next record
*/
public void addIterationSink(String componentName, String iterationTail, String iterationID,
- int parallelism, String directName) {
+ int parallelism, long waitTime) {
- addComponent(componentName, StreamIterationSink.class, null, null, null, parallelism);
+ addComponent(componentName, StreamIterationSink.class, null, null, null, null, parallelism);
iterationIds.put(componentName, iterationID);
+ iterationIDtoSinkName.put(iterationID, componentName);
setBytesFrom(iterationTail, componentName);
-
- if (directName != null) {
- setUserDefinedName(componentName, directName);
- } else {
- setUserDefinedName(componentName, "iterate");
- }
+ iterationWaitTime.put(iterationIDtoSinkName.get(iterationID), waitTime);
if (LOG.isDebugEnabled()) {
LOG.debug("ITERATION SINK: " + componentName);
@@ -281,6 +312,8 @@ public class JobGraphBuilder {
* Name of the component
* @param componentClass
* The class of the vertex
+ * @param typeWrapper
+ * Wrapper of the types for serialization
* @param invokableObject
* The user defined invokable object
* @param operatorName
@@ -292,18 +325,22 @@ public class JobGraphBuilder {
*/
private void addComponent(String componentName,
Class<? extends AbstractInvokable> componentClass,
- StreamComponentInvokable invokableObject, String operatorName,
+ TypeSerializerWrapper<?, ?, ?> typeWrapper,
+ StreamComponentInvokable<?> invokableObject, String operatorName,
byte[] serializedFunction, int parallelism) {
componentClasses.put(componentName, componentClass);
+ typeWrappers.put(componentName, typeWrapper);
setParallelism(componentName, parallelism);
+ mutability.put(componentName, false);
invokableObjects.put(componentName, invokableObject);
operatorNames.put(componentName, operatorName);
serializedFunctions.put(componentName, serializedFunction);
outEdgeList.put(componentName, new ArrayList<String>());
outEdgeType.put(componentName, new ArrayList<Integer>());
+ outEdgeNames.put(componentName, new ArrayList<List<String>>());
inEdgeList.put(componentName, new ArrayList<String>());
- connectionTypes.put(componentName, new ArrayList<StreamPartitioner<? extends Tuple>>());
+ connectionTypes.put(componentName, new ArrayList<StreamPartitioner<?>>());
iterationTailCount.put(componentName, 0);
}
@@ -318,12 +355,11 @@ public class JobGraphBuilder {
// Get vertex attributes
Class<? extends AbstractInvokable> componentClass = componentClasses.get(componentName);
- StreamComponentInvokable invokableObject = invokableObjects.get(componentName);
+ StreamComponentInvokable<?> invokableObject = invokableObjects.get(componentName);
String operatorName = operatorNames.get(componentName);
byte[] serializedFunction = serializedFunctions.get(componentName);
int parallelism = componentParallelism.get(componentName);
byte[] outputSelector = outputSelectors.get(componentName);
- String userDefinedName = userDefinedNames.get(componentName);
// Create vertex object
AbstractJobVertex component = null;
@@ -336,6 +372,8 @@ public class JobGraphBuilder {
} else if (componentClass.equals(StreamSink.class)
|| componentClass.equals(StreamIterationSink.class)) {
component = new JobOutputVertex(componentName, this.jobGraph);
+ } else {
+ throw new RuntimeException("Unsupported component class");
}
component.setInvokableClass(componentClass);
@@ -344,31 +382,21 @@ public class JobGraphBuilder {
LOG.debug("Parallelism set: " + parallelism + " for " + componentName);
}
- Configuration config = component.getConfiguration();
+ StreamConfig config = new StreamConfig(component.getConfiguration());
+ config.setMutability(mutability.get(componentName));
+ config.setBufferTimeout(bufferTimeout.get(componentName));
+ config.setTypeWrapper(typeWrappers.get(componentName));
// Set vertex config
- if (invokableObject != null) {
- config.setClass("userfunction", invokableObject.getClass());
- addSerializedObject(invokableObject, config);
- }
- config.setString("componentName", componentName);
- if (serializedFunction != null) {
- config.setBytes("operator", serializedFunction);
- config.setString("operatorName", operatorName);
- }
-
- if (userDefinedName != null) {
- config.setString("userDefinedName", userDefinedName);
- }
-
- if (outputSelector != null) {
- config.setBoolean("directedEmit", true);
- config.setBytes("outputSelector", outputSelector);
- }
+ config.setUserInvokable(invokableObject);
+ config.setComponentName(componentName);
+ config.setFunction(serializedFunction, operatorName);
+ config.setOutputSelector(outputSelector);
if (componentClass.equals(StreamIterationSource.class)
|| componentClass.equals(StreamIterationSink.class)) {
- config.setString("iteration-id", iterationIds.get(componentName));
+ config.setIterationId(iterationIds.get(componentName));
+ config.setIterationWaitTime(iterationWaitTime.get(componentName));
}
components.put(componentName, component);
@@ -380,51 +408,6 @@ public class JobGraphBuilder {
}
/**
- * Adds serialized invokable object to the JobVertex configuration
- *
- * @param invokableObject
- * Invokable object to serialize
- * @param config
- * JobVertex configuration to which the serialized invokable will
- * be added
- */
- private void addSerializedObject(Serializable invokableObject, Configuration config) {
-
- ByteArrayOutputStream baos = null;
- ObjectOutputStream oos = null;
- try {
- baos = new ByteArrayOutputStream();
-
- oos = new ObjectOutputStream(baos);
-
- oos.writeObject(invokableObject);
-
- config.setBytes("serializedudf", baos.toByteArray());
- } catch (Exception e) {
- throw new RuntimeException("Cannot serialize invokable object "
- + invokableObject.getClass(), e);
- }
-
- }
-
- /**
- * Sets the user defined name for the selected component
- *
- * @param componentName
- * Name of the component for which the user defined name will be
- * set
- * @param userDefinedName
- * User defined name to set for the component
- */
- public void setUserDefinedName(String componentName, String userDefinedName) {
- userDefinedNames.put(componentName, userDefinedName);
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Name set: " + userDefinedName + " for " + componentName);
- }
- }
-
- /**
* Sets the number of parallel instances created for the given component.
*
* @param componentName
@@ -436,6 +419,14 @@ public class JobGraphBuilder {
componentParallelism.put(componentName, parallelism);
}
+ public void setMutability(String componentName, boolean isMutable) {
+ mutability.put(componentName, isMutable);
+ }
+
+ public void setBufferTimeout(String componentName, long bufferTimeout) {
+ this.bufferTimeout.put(componentName, bufferTimeout);
+ }
+
/**
* Connects two vertices in the JobGraph using the selected partitioner
* settings
@@ -448,13 +439,16 @@ public class JobGraphBuilder {
* Partitioner object
* @param typeNumber
* Number of the type (used at co-functions)
+ * @param outputNames
+ * User defined names of the out edge
*/
public void setEdge(String upStreamComponentName, String downStreamComponentName,
- StreamPartitioner<? extends Tuple> partitionerObject, int typeNumber) {
+ StreamPartitioner<?> partitionerObject, int typeNumber, List<String> outputNames) {
outEdgeList.get(upStreamComponentName).add(downStreamComponentName);
outEdgeType.get(upStreamComponentName).add(typeNumber);
inEdgeList.get(downStreamComponentName).add(upStreamComponentName);
connectionTypes.get(upStreamComponentName).add(partitionerObject);
+ outEdgeNames.get(upStreamComponentName).add(outputNames);
}
/**
@@ -468,13 +462,13 @@ public class JobGraphBuilder {
* @param partitionerObject
* The partitioner
*/
- private <T extends Tuple> void connect(String upStreamComponentName,
- String downStreamComponentName, StreamPartitioner<T> partitionerObject) {
+ private <T> void connect(String upStreamComponentName, String downStreamComponentName,
+ StreamPartitioner<T> partitionerObject) {
AbstractJobVertex upStreamComponent = components.get(upStreamComponentName);
AbstractJobVertex downStreamComponent = components.get(downStreamComponentName);
- Configuration config = upStreamComponent.getConfiguration();
+ StreamConfig config = new StreamConfig(upStreamComponent.getConfiguration());
try {
if (partitionerObject.getClass().equals(ForwardPartitioner.class)) {
@@ -497,49 +491,25 @@ public class JobGraphBuilder {
int outputIndex = upStreamComponent.getNumberOfForwardConnections() - 1;
- putOutputNameToConfig(upStreamComponentName, downStreamComponentName, outputIndex, config);
-
- config.setBytes("partitionerObject_" + outputIndex,
- SerializationUtils.serialize(partitionerObject));
-
- config.setInteger("numOfOutputs_" + outputIndex,
+ config.setOutputName(outputIndex, outEdgeNames.get(upStreamComponentName).get(outputIndex));
+ config.setPartitioner(outputIndex, partitionerObject);
+ config.setNumberOfOutputChannels(outputIndex,
componentParallelism.get(downStreamComponentName));
-
}
/**
- * Sets the user defined name for an output edge in the graph
- *
- * @param upStreamComponentName
- * The name of the component to which the output name will be set
- * @param downStreamComponentName
- * The name of the component representing the output
- * @param index
- * Index of the output channel
- * @param config
- * Config of the upstream component
- */
- private void putOutputNameToConfig(String upStreamComponentName,
- String downStreamComponentName, int index, Configuration config) {
-
- String outputName = userDefinedNames.get(downStreamComponentName);
- if (outputName != null) {
- config.setString("outputName_" + (index), outputName);
- }
- }
-
- /**
- * Sets the parallelism of the iteration head of the given iteration id to
- * the parallelism given.
+ * Sets the parallelism and buffertimeout of the iteration head of the given
+ * iteration id to the parallelism given.
*
* @param iterationID
* ID of the iteration
- * @param parallelism
- * Parallelism to set, typically the parallelism of the iteration
- * tail.
+ * @param iterationTail
+ * ID of the iteration tail
*/
- public void setIterationSourceParallelism(String iterationID, int parallelism) {
- setParallelism(iterationHeadNames.get(iterationID), parallelism);
+ public void setIterationSourceSettings(String iterationID, String iterationTail) {
+ setParallelism(iterationIDtoSourceName.get(iterationID),
+ componentParallelism.get(iterationTail));
+ setBufferTimeout(iterationIDtoSourceName.get(iterationID), bufferTimeout.get(iterationTail));
}
/**
@@ -552,8 +522,7 @@ public class JobGraphBuilder {
* @param serializedOutputSelector
* Byte array representing the serialized output selector.
*/
- public <T extends Tuple> void setOutputSelector(String componentName,
- byte[] serializedOutputSelector) {
+ public <T> void setOutputSelector(String componentName, byte[] serializedOutputSelector) {
outputSelectors.put(componentName, serializedOutputSelector);
if (LOG.isDebugEnabled()) {
@@ -563,7 +532,8 @@ public class JobGraphBuilder {
}
/**
- * Sets udf operator from one component to another, used with some sinks.
+ * Sets udf operator and TypeSerializerWrapper from one component to
+ * another, used with some sinks.
*
* @param from
* from
@@ -574,7 +544,7 @@ public class JobGraphBuilder {
operatorNames.put(to, operatorNames.get(from));
serializedFunctions.put(to, serializedFunctions.get(from));
-
+ typeWrappers.put(to, typeWrappers.get(from));
}
/**
@@ -600,7 +570,7 @@ public class JobGraphBuilder {
AbstractJobVertex maxParallelismVertex = components.get(maxParallelismVertexName);
for (String componentName : components.keySet()) {
- if (componentName != maxParallelismVertexName) {
+ if (!componentName.equals(maxParallelismVertexName)) {
components.get(componentName).setVertexToShareInstancesWith(maxParallelismVertex);
}
}
@@ -612,8 +582,8 @@ public class JobGraphBuilder {
*/
private void setNumberOfJobInputs() {
for (AbstractJobVertex component : components.values()) {
- component.getConfiguration().setInteger("numberOfInputs",
- component.getNumberOfBackwardConnections());
+ (new StreamConfig(component.getConfiguration())).setNumberOfInputs(component
+ .getNumberOfBackwardConnections());
}
}
@@ -623,8 +593,8 @@ public class JobGraphBuilder {
*/
private void setNumberOfJobOutputs() {
for (AbstractJobVertex component : components.values()) {
- component.getConfiguration().setInteger("numberOfOutputs",
- component.getNumberOfForwardConnections());
+ (new StreamConfig(component.getConfiguration())).setNumberOfOutputs(component
+ .getNumberOfForwardConnections());
}
}
@@ -641,16 +611,16 @@ public class JobGraphBuilder {
for (String upStreamComponentName : outEdgeList.keySet()) {
int i = 0;
- ArrayList<Integer> outEdgeTypeList = outEdgeType.get(upStreamComponentName);
+ List<Integer> outEdgeTypeList = outEdgeType.get(upStreamComponentName);
for (String downStreamComponentName : outEdgeList.get(upStreamComponentName)) {
- Configuration downStreamComponentConfig = components.get(downStreamComponentName)
- .getConfiguration();
+ StreamConfig downStreamComponentConfig = new StreamConfig(components.get(
+ downStreamComponentName).getConfiguration());
- int inputNumber = downStreamComponentConfig.getInteger("numberOfInputs", 0);
- downStreamComponentConfig.setInteger("inputType_" + inputNumber++,
- outEdgeTypeList.get(i));
- downStreamComponentConfig.setInteger("numberOfInputs", inputNumber);
+ int inputNumber = downStreamComponentConfig.getNumberOfInputs();
+
+ downStreamComponentConfig.setInputType(inputNumber++, outEdgeTypeList.get(i));
+ downStreamComponentConfig.setNumberOfInputs(inputNumber);
connect(upStreamComponentName, downStreamComponentName,
connectionTypes.get(upStreamComponentName).get(i));
@@ -661,7 +631,6 @@ public class JobGraphBuilder {
setAutomaticInstanceSharing();
setNumberOfJobInputs();
setNumberOfJobOutputs();
-
}
/**
@@ -674,4 +643,4 @@ public class JobGraphBuilder {
return jobGraph;
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a2c4137f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index 35cfc24..f56614d 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -127,6 +127,10 @@ public abstract class StreamExecutionEnvironment {
protected void setMutability(DataStream<?> stream, boolean isMutable) {
jobGraphBuilder.setMutability(stream.getId(), isMutable);
}
+
+ protected void setBufferTimeout(DataStream<?> stream, long bufferTimeout) {
+ jobGraphBuilder.setBufferTimeout(stream.getId(), bufferTimeout);
+ }
/**
* Sets the number of hardware contexts (CPU cores / threads) used when
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a2c4137f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index 1a51492..8afbddf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -48,7 +48,7 @@ import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
public abstract class AbstractStreamComponent<OUT extends Tuple> extends AbstractInvokable {
-
+
private static final Log LOG = LogFactory.getLog(AbstractStreamComponent.class);
protected TupleTypeInfo<OUT> outTupleTypeInfo = null;
@@ -89,7 +89,6 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
}
return collector;
}
-
@SuppressWarnings({ "rawtypes", "unchecked" })
protected void setSerializer(Object function, Class<?> clazz, int typeParameter) {
@@ -99,9 +98,9 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
outTupleSerializer = new StreamRecordSerializer(outTupleTypeInfo.createSerializer());
outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(outTupleSerializer);
}
-
- protected void setConfigOutputs(List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
+ protected void setConfigOutputs(
+ List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
int numberOfOutputs = configuration.getInteger("numberOfOutputs", 0);
@@ -109,7 +108,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
setPartitioner(i, outputs);
}
}
-
+
private void setPartitioner(int outputNumber,
List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
@@ -120,8 +119,18 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
try {
outputPartitioner = deserializeObject(serializedPartitioner);
- RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output = new RecordWriter<SerializationDelegate<StreamRecord<OUT>>>(
- this, outputPartitioner);
+ RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output;
+
+ long bufferTimeout = configuration.getLong("bufferTimeout", 0);
+
+ if (bufferTimeout > 0) {
+ output = new StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>>(this,
+ outputPartitioner, bufferTimeout);
+ } else {
+ output = new RecordWriter<SerializationDelegate<StreamRecord<OUT>>>(this,
+ outputPartitioner);
+ }
+
outputs.add(output);
String outputName = configuration.getString("outputName_" + outputNumber, null);
@@ -134,12 +143,12 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
+ " with " + outputNumber + " outputs");
}
} catch (Exception e) {
- throw new StreamComponentException("Cannot deserialize " + outputPartitioner.getClass().getSimpleName() + " of " +
- name + " with " + outputNumber
- + " outputs", e);
+ throw new StreamComponentException("Cannot deserialize "
+ + outputPartitioner.getClass().getSimpleName() + " of " + name + " with "
+ + outputNumber + " outputs", e);
}
}
-
+
/**
* Reads and creates a StreamComponent from the config.
*
@@ -166,9 +175,9 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
return userFunction;
}
-
- protected <IN extends Tuple> MutableObjectIterator<StreamRecord<IN>> createInputIterator(MutableReader<?> inputReader,
- TypeSerializer<?> serializer) {
+
+ protected <IN extends Tuple> MutableObjectIterator<StreamRecord<IN>> createInputIterator(
+ MutableReader<?> inputReader, TypeSerializer<?> serializer) {
// generic data type serialization
@SuppressWarnings("unchecked")
@@ -177,13 +186,13 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
final MutableObjectIterator<StreamRecord<IN>> iter = new ReaderIterator(reader, serializer);
return iter;
}
-
+
@SuppressWarnings("unchecked")
protected static <T> T deserializeObject(byte[] serializedObject) throws IOException,
ClassNotFoundException {
return (T) SerializationUtils.deserialize(serializedObject);
}
-
+
protected abstract void setInvokable();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a2c4137f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
new file mode 100755
index 0000000..a89935a
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
@@ -0,0 +1,115 @@
+package org.apache.flink.streaming.api.streamcomponent;
+
+import java.io.IOException;
+
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.runtime.io.network.Buffer;
+import org.apache.flink.runtime.io.network.api.ChannelSelector;
+import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.io.network.api.RoundRobinChannelSelector;
+import org.apache.flink.runtime.io.network.bufferprovider.BufferProvider;
+import org.apache.flink.runtime.io.network.serialization.RecordSerializer;
+import org.apache.flink.runtime.io.network.serialization.SpanningRecordSerializer;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+
+public class StreamRecordWriter<T extends IOReadableWritable> extends RecordWriter<T> {
+
+ private final BufferProvider bufferPool;
+
+ private final ChannelSelector<T> channelSelector;
+
+ private int numChannels;
+
+ private long timeout;
+
+ /** RecordSerializer per outgoing channel */
+ private RecordSerializer<T>[] serializers;
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public StreamRecordWriter(AbstractInvokable invokable) {
+ this(invokable, new RoundRobinChannelSelector<T>(), 1000);
+ }
+
+ public StreamRecordWriter(AbstractInvokable invokable, ChannelSelector<T> channelSelector) {
+ this(invokable, channelSelector, 1000);
+ }
+
+ public StreamRecordWriter(AbstractInvokable invokable, ChannelSelector<T> channelSelector,
+ long timeout) {
+ // initialize the gate
+ super(invokable);
+
+ this.timeout = timeout;
+ this.bufferPool = invokable.getEnvironment().getOutputBufferProvider();
+ this.channelSelector = channelSelector;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void initializeSerializers() {
+ this.numChannels = this.outputGate.getNumChannels();
+ this.serializers = new RecordSerializer[numChannels];
+ for (int i = 0; i < this.numChannels; i++) {
+ this.serializers[i] = new SpanningRecordSerializer<T>();
+ }
+ (new OutputFlusher()).start();
+ }
+
+ @Override
+ public void emit(final T record) throws IOException, InterruptedException {
+ for (int targetChannel : this.channelSelector.selectChannels(record, this.numChannels)) {
+ // serialize with corresponding serializer and send full buffer
+
+ RecordSerializer<T> serializer = this.serializers[targetChannel];
+
+ synchronized (serializer) {
+ RecordSerializer.SerializationResult result = serializer.addRecord(record);
+ while (result.isFullBuffer()) {
+ Buffer buffer = serializer.getCurrentBuffer();
+ if (buffer != null) {
+ sendBuffer(buffer, targetChannel);
+ }
+
+ buffer = this.bufferPool.requestBufferBlocking(this.bufferPool.getBufferSize());
+ result = serializer.setNextBuffer(buffer);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void flush() throws IOException, InterruptedException {
+ for (int targetChannel = 0; targetChannel < this.numChannels; targetChannel++) {
+ RecordSerializer<T> serializer = this.serializers[targetChannel];
+ synchronized (serializer) {
+ Buffer buffer = serializer.getCurrentBuffer();
+ if (buffer != null) {
+ sendBuffer(buffer, targetChannel);
+ }
+
+ serializer.clear();
+ }
+
+ }
+ }
+
+ private class OutputFlusher extends Thread {
+
+ @Override
+ public void run() {
+ while (!outputGate.isClosed()) {
+ try {
+ Thread.sleep(timeout);
+ flush();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+
+ }
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a2c4137f/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
index 1c43a66..67dce9d 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
@@ -19,57 +19,15 @@
package org.apache.flink.streaming.api;
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.streaming.util.LogUtils;
-import org.apache.flink.util.Collector;
import org.apache.log4j.Level;
import org.junit.Test;
public class PrintTest {
- public static final class MyFlatMap extends
- FlatMapFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public void flatMap(Tuple2<Integer, String> value, Collector<Tuple2<Integer, String>> out)
- throws Exception {
- out.collect(new Tuple2<Integer, String>(value.f0 * value.f0, value.f1));
-
- }
-
- }
-
+
private static final long MEMORYSIZE = 32;
- public static final class Increment extends FlatMapFunction<Tuple1<Integer>, Tuple1<Integer>> {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public void flatMap(Tuple1<Integer> value, Collector<Tuple1<Integer>> out) throws Exception {
- if (value.f0 < 5) {
- out.collect(new Tuple1<Integer>(value.f0 + 1));
- }
-
- }
-
- }
-
- public static final class Forward extends FlatMapFunction<Tuple1<Integer>, Tuple1<Integer>> {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public void flatMap(Tuple1<Integer> value, Collector<Tuple1<Integer>> out) throws Exception {
- out.collect(value);
-
- }
-
- }
@Test
public void test() throws Exception {
@@ -82,7 +40,4 @@ public class PrintTest {
}
-
-
-
}
[13/51] [abbrv] git commit: [streaming] Automerge error + License fix
Posted by se...@apache.org.
[streaming] Automerge error + License fix
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/f186f3d5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/f186f3d5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/f186f3d5
Branch: refs/heads/master
Commit: f186f3d5144cc52f11562a069ee275df96fbf0e0
Parents: 2c4e195
Author: gyfora <gy...@gmail.com>
Authored: Tue Jul 22 19:46:55 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:19:17 2014 +0200
----------------------------------------------------------------------
.../flink/streaming/api/JobGraphBuilder.java | 26 ++++++++++--------
.../api/StreamExecutionEnvironment.java | 2 +-
.../api/invokable/operator/BatchIterator.java | 19 +++++++++++++
.../api/invokable/operator/CoMapTest.java | 29 ++++++++++++--------
4 files changed, 52 insertions(+), 24 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f186f3d5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index bf9057b..986172b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -574,12 +574,12 @@ public class JobGraphBuilder {
setEdge(upStreamComponentName, downStreamComponentName, new ForwardPartitioner<T>(),
typeNumber);
}
-
+
/**
* Connects two components with the given names by distribute partitioning.
* <p>
- * Distribute partitioning: sends the output tuples evenly distributed
- * along the selected channels
+ * Distribute partitioning: sends the output tuples evenly distributed along
+ * the selected channels
*
* @param inputStream
* The DataStream object of the input
@@ -587,10 +587,13 @@ public class JobGraphBuilder {
* Name of the upstream component, that will emit the tuples
* @param downStreamComponentName
* Name of the downstream component, that will receive the tuples
+ * @param typeNumber
+ * Number of the type (used at co-functions)
*/
public <T extends Tuple> void distributeConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName) {
- setEdge(upStreamComponentName, downStreamComponentName, new DistributePartitioner<T>());
+ String upStreamComponentName, String downStreamComponentName, int typeNumber) {
+ setEdge(upStreamComponentName, downStreamComponentName, new DistributePartitioner<T>(),
+ typeNumber);
}
/**
@@ -773,20 +776,21 @@ public class JobGraphBuilder {
for (String componentName : outEdgeList.keySet()) {
createVertex(componentName);
}
-
+
for (String upStreamComponentName : outEdgeList.keySet()) {
int i = 0;
-
+
ArrayList<Integer> outEdgeTypeList = outEdgeType.get(upStreamComponentName);
for (String downStreamComponentName : outEdgeList.get(upStreamComponentName)) {
Configuration downStreamComponentConfig = components.get(downStreamComponentName)
.getConfiguration();
-
- int inputNumber = downStreamComponentConfig.getInteger("numberOfInputs", 0);
- downStreamComponentConfig.setInteger("inputType_" + inputNumber++, outEdgeTypeList.get(i));
+
+ int inputNumber = downStreamComponentConfig.getInteger("numberOfInputs", 0);
+ downStreamComponentConfig.setInteger("inputType_" + inputNumber++,
+ outEdgeTypeList.get(i));
downStreamComponentConfig.setInteger("numberOfInputs", inputNumber);
-
+
connect(upStreamComponentName, downStreamComponentName,
connectionTypes.get(upStreamComponentName).get(i));
i++;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f186f3d5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index b69c457..4b5bc98 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -572,7 +572,7 @@ public abstract class StreamExecutionEnvironment {
jobGraphBuilder.forwardConnect(inputStream, input, outputID, typeNumber);
break;
case DISTRIBUTE:
- jobGraphBuilder.distributeConnect(inputStream, input, outputID);
+ jobGraphBuilder.distributeConnect(inputStream, input, outputID, typeNumber);
break;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f186f3d5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
index 1432749..7b971b9 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
@@ -1,3 +1,22 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
package org.apache.flink.streaming.api.invokable.operator;
import java.util.Iterator;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f186f3d5/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
index b2fd3cf..ecd81bd 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
@@ -40,6 +40,16 @@ public class CoMapTest implements Serializable {
private static Set<String> result;
private static Set<String> expected = new HashSet<String>();
+ public CoMapTest() {
+ expected.add("a");
+ expected.add("b");
+ expected.add("c");
+ expected.add("1");
+ expected.add("2");
+ expected.add("3");
+ expected.add("4");
+ }
+
private final static class EmptySink extends SinkFunction<Tuple1<Boolean>> {
private static final long serialVersionUID = 1L;
@@ -68,16 +78,8 @@ public class CoMapTest implements Serializable {
@Test
public void test() {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
-
+
result = new HashSet<String>();
-
- expected.add("a");
- expected.add("b");
- expected.add("c");
- expected.add("1");
- expected.add("2");
- expected.add("3");
- expected.add("4");
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
@@ -93,6 +95,8 @@ public class CoMapTest implements Serializable {
@Test
public void multipleInputTest() {
+ LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+
result = new HashSet<String>();
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
@@ -102,11 +106,12 @@ public class CoMapTest implements Serializable {
DataStream<Tuple1<Integer>> ds2 = env.fromElements(2, 4).connectWith(ds1);
DataStream<Tuple1<String>> ds3 = env.fromElements("a", "b");
-
+
@SuppressWarnings({ "unused", "unchecked" })
- DataStream<Tuple1<Boolean>> ds4 = env.fromElements("c").connectWith(ds3).coMapWith(new MyCoMap(),
+ DataStream<Tuple1<Boolean>> ds4 = env.fromElements("c").connectWith(ds3)
+ .coMapWith(new MyCoMap(),
- ds2).addSink(new EmptySink());
+ ds2).addSink(new EmptySink());
env.executeTest(32);
Assert.assertArrayEquals(expected.toArray(), result.toArray());
[11/51] [abbrv] git commit: [streaming] CoMapTest update
Posted by se...@apache.org.
[streaming] CoMapTest update
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/1edd0317
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/1edd0317
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/1edd0317
Branch: refs/heads/master
Commit: 1edd0317d3b77f1ddebf3b91c89cbd12455eeca0
Parents: f186f3d
Author: gyfora <gy...@gmail.com>
Authored: Tue Jul 22 22:32:56 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:19:17 2014 +0200
----------------------------------------------------------------------
.../api/invokable/operator/CoMapTest.java | 37 +++++---------------
1 file changed, 8 insertions(+), 29 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1edd0317/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
index ecd81bd..d3cbcbc 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
@@ -40,16 +40,6 @@ public class CoMapTest implements Serializable {
private static Set<String> result;
private static Set<String> expected = new HashSet<String>();
- public CoMapTest() {
- expected.add("a");
- expected.add("b");
- expected.add("c");
- expected.add("1");
- expected.add("2");
- expected.add("3");
- expected.add("4");
- }
-
private final static class EmptySink extends SinkFunction<Tuple1<Boolean>> {
private static final long serialVersionUID = 1L;
@@ -76,26 +66,15 @@ public class CoMapTest implements Serializable {
}
@Test
- public void test() {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
-
- result = new HashSet<String>();
-
- LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
- DataStream<Tuple1<Integer>> ds1 = env.fromElements(1, 2, 3, 4);
-
- @SuppressWarnings("unused")
- DataStream<Tuple1<Boolean>> ds2 = env.fromElements("a", "b", "c")
- .coMapWith(new MyCoMap(), ds1).addSink(new EmptySink());
-
- env.executeTest(32);
- Assert.assertArrayEquals(expected.toArray(), result.toArray());
- }
-
- @Test
public void multipleInputTest() {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+ expected.add("a");
+ expected.add("b");
+ expected.add("c");
+ expected.add("1");
+ expected.add("2");
+ expected.add("3");
+ expected.add("4");
result = new HashSet<String>();
@@ -114,6 +93,6 @@ public class CoMapTest implements Serializable {
ds2).addSink(new EmptySink());
env.executeTest(32);
- Assert.assertArrayEquals(expected.toArray(), result.toArray());
+ Assert.assertEquals(expected, result);
}
}
[28/51] [abbrv] [streaming] Added support for simple types instead of
Tuple1 in the API
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java
index a42b0bb..0a88efd 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java
@@ -21,9 +21,7 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
-import org.apache.flink.api.java.tuple.Tuple;
-
-public abstract class UserTaskInvokable<IN extends Tuple, OUT extends Tuple> extends
+public abstract class UserTaskInvokable<IN, OUT> extends
StreamRecordInvokable<IN, OUT> implements Serializable {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 4aa540c..46e79de 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -23,9 +23,8 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.java.functions.GroupReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
+public class BatchReduceInvokable<IN, OUT> extends
StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private int batchSize;
@@ -53,7 +52,7 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
}
}
counter++;
- tupleBatch.add(reuse.getTuple());
+ tupleBatch.add(reuse.getObject());
resetReuse();
} while (counter < batchSize);
reducer.reduce(tupleBatch.iterator(), collector);
@@ -99,7 +98,7 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
if (hasNext()) {
counter++;
loadedNext = false;
- return reuse.getTuple();
+ return reuse.getObject();
} else {
counter++;
loadedNext = false;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
index edeb79a..f2b2930 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
@@ -20,10 +20,9 @@
package org.apache.flink.streaming.api.invokable.operator;
import org.apache.flink.api.java.functions.FilterFunction;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-public class FilterInvokable<IN extends Tuple> extends UserTaskInvokable<IN, IN> {
+public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
private static final long serialVersionUID = 1L;
@@ -35,9 +34,9 @@ public class FilterInvokable<IN extends Tuple> extends UserTaskInvokable<IN, IN>
@Override
protected void immutableInvoke() throws Exception {
- while (recordIterator.next(reuse) != null) {
- if (filterFunction.filter(reuse.getTuple())) {
- collector.collect(reuse.getTuple());
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ if (filterFunction.filter(reuse.getObject())) {
+ collector.collect(reuse.getObject());
}
resetReuse();
}
@@ -45,9 +44,9 @@ public class FilterInvokable<IN extends Tuple> extends UserTaskInvokable<IN, IN>
@Override
protected void mutableInvoke() throws Exception {
- while (recordIterator.next(reuse) != null) {
- if (filterFunction.filter(reuse.getTuple())) {
- collector.collect(reuse.getTuple());
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ if (filterFunction.filter(reuse.getObject())) {
+ collector.collect(reuse.getObject());
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
index 279b160..11e7853 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
@@ -20,11 +20,9 @@
package org.apache.flink.streaming.api.invokable.operator;
import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-public class FlatMapInvokable<IN extends Tuple, OUT extends Tuple> extends
- UserTaskInvokable<IN, OUT> {
+public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private FlatMapFunction<IN, OUT> flatMapper;
@@ -35,16 +33,16 @@ public class FlatMapInvokable<IN extends Tuple, OUT extends Tuple> extends
@Override
protected void immutableInvoke() throws Exception {
- while (recordIterator.next(reuse) != null) {
- flatMapper.flatMap(reuse.getTuple(), collector);
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ flatMapper.flatMap(reuse.getObject(), collector);
resetReuse();
}
}
@Override
protected void mutableInvoke() throws Exception {
- while (recordIterator.next(reuse) != null) {
- flatMapper.flatMap(reuse.getTuple(), collector);
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ flatMapper.flatMap(reuse.getObject(), collector);
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
index 3c56b6f..794d765 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
@@ -20,10 +20,9 @@
package org.apache.flink.streaming.api.invokable.operator;
import org.apache.flink.api.java.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-public class MapInvokable<IN extends Tuple, OUT extends Tuple> extends UserTaskInvokable<IN, OUT> {
+public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private MapFunction<IN, OUT> mapper;
@@ -34,16 +33,16 @@ public class MapInvokable<IN extends Tuple, OUT extends Tuple> extends UserTaskI
@Override
protected void immutableInvoke() throws Exception {
- while (recordIterator.next(reuse) != null) {
- collector.collect(mapper.map(reuse.getTuple()));
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ collector.collect(mapper.map(reuse.getObject()));
resetReuse();
}
}
@Override
protected void mutableInvoke() throws Exception {
- while (recordIterator.next(reuse) != null) {
- collector.collect(mapper.map(reuse.getTuple()));
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ collector.collect(mapper.map(reuse.getObject()));
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
index f7ea566..a574ebc 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
@@ -20,10 +20,9 @@
package org.apache.flink.streaming.api.invokable.operator;
import org.apache.flink.api.java.functions.GroupReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-public abstract class StreamReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
+public abstract class StreamReduceInvokable<IN, OUT> extends
UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
protected GroupReduceFunction<IN, OUT> reducer;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index 67c15dc..7710bd8 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -23,9 +23,8 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.java.functions.GroupReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-public class WindowReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
+public class WindowReduceInvokable<IN, OUT> extends
StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private long windowSize;
@@ -54,7 +53,7 @@ public class WindowReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
break;
}
}
- tupleBatch.add(reuse.getTuple());
+ tupleBatch.add(reuse.getObject());
resetReuse();
} while (System.currentTimeMillis() - startTime < windowSize);
reducer.reduce(tupleBatch.iterator(), collector);
@@ -99,10 +98,10 @@ public class WindowReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
public IN next() {
if (hasNext()) {
loadedNext = false;
- return reuse.getTuple();
+ return reuse.getObject();
} else {
loadedNext = false;
- return reuse.getTuple();
+ return reuse.getObject();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
index 884e361..d854e89 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
@@ -19,14 +19,13 @@
package org.apache.flink.streaming.api.invokable.operator.co;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
-public abstract class CoInvokable<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
+public abstract class CoInvokable<IN1, IN2, OUT> extends
StreamComponentInvokable<OUT> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
index be5c42f..e899367 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
@@ -19,11 +19,9 @@
package org.apache.flink.streaming.api.invokable.operator.co;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
-public class CoMapInvokable<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
- CoInvokable<IN1, IN2, OUT> {
+public class CoMapInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
private CoMapFunction<IN1, IN2, OUT> mapper;
@@ -39,14 +37,14 @@ public class CoMapInvokable<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tu
boolean noMoreRecordOnInput2 = false;
do {
- noMoreRecordOnInput1 = recordIterator1.next(reuse1) == null;
+ noMoreRecordOnInput1 = ((reuse1 = recordIterator1.next(reuse1)) == null);
if (!noMoreRecordOnInput1) {
- collector.collect(mapper.map1(reuse1.getTuple()));
+ collector.collect(mapper.map1(reuse1.getObject()));
}
- noMoreRecordOnInput2 = recordIterator2.next(reuse2) == null;
+ noMoreRecordOnInput2 = ((reuse2 = recordIterator2.next(reuse2)) == null);
if (!noMoreRecordOnInput2) {
- collector.collect(mapper.map2(reuse2.getTuple()));
+ collector.collect(mapper.map2(reuse2.getObject()));
}
if (!this.isMutable) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index fcf87e2..e50803f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -26,8 +26,6 @@ import org.apache.commons.lang.SerializationUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
@@ -43,21 +41,22 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.partitioner.StreamPartitioner;
import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.types.TypeInformation;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
-public abstract class AbstractStreamComponent<OUT extends Tuple> extends AbstractInvokable {
+public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
protected static final String SOURCE = "source";
private static final Log LOG = LogFactory.getLog(AbstractStreamComponent.class);
- protected TupleTypeInfo<OUT> outTupleTypeInfo = null;
- protected StreamRecordSerializer<OUT> outTupleSerializer = null;
+ protected TypeInformation<OUT> outTypeInfo = null;
+ protected StreamRecordSerializer<OUT> outSerializer = null;
protected SerializationDelegate<StreamRecord<OUT>> outSerializationDelegate = null;
protected StreamConfig configuration;
- protected TypeSerializerWrapper<? extends Tuple, ? extends Tuple, OUT> typeWrapper;
+ protected TypeSerializerWrapper<?, ?, OUT> typeWrapper;
protected StreamCollector<OUT> collector;
protected int instanceID;
protected String name;
@@ -105,9 +104,9 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
}
protected void setSerializer() {
- outTupleTypeInfo = typeWrapper.getOutputTupleTypeInfo();
- outTupleSerializer = new StreamRecordSerializer<OUT>(outTupleTypeInfo.createSerializer());
- outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(outTupleSerializer);
+ outTypeInfo = typeWrapper.getOutputTypeInfo();
+ outSerializer = new StreamRecordSerializer<OUT>(outTypeInfo);
+ outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(outSerializer);
}
protected void setConfigOutputs(
@@ -171,7 +170,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
return (T) configuration.getUserInvokableObject();
}
- protected <IN extends Tuple> MutableObjectIterator<StreamRecord<IN>> createInputIterator(
+ protected <IN> MutableObjectIterator<StreamRecord<IN>> createInputIterator(
MutableReader<?> inputReader, TypeSerializer<?> serializer) {
// generic data type serialization
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
index 0c02c16..ede30b4 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -27,7 +27,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.common.functions.AbstractFunction;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.MutableRecordReader;
@@ -38,14 +37,15 @@ import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+import org.apache.flink.types.TypeInformation;
import org.apache.flink.util.MutableObjectIterator;
public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
AbstractStreamComponent<OUT> {
private static final Log LOG = LogFactory.getLog(CoStreamTask.class);
- protected StreamRecordSerializer<IN1> inTupleDeserializer1 = null;
- protected StreamRecordSerializer<IN2> inTupleDeserializer2 = null;
+ protected StreamRecordSerializer<IN1> inputDeserializer1 = null;
+ protected StreamRecordSerializer<IN2> inputDeserializer2 = null;
private MutableReader<IOReadableWritable> inputs1;
private MutableReader<IOReadableWritable> inputs2;
@@ -83,11 +83,11 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
@SuppressWarnings({ "unchecked", "rawtypes" })
private void setDeserializers(Object function, Class<? extends AbstractFunction> clazz) {
- TupleTypeInfo<IN1> inTupleTypeInfo = (TupleTypeInfo<IN1>) typeWrapper.getInputTupleTypeInfo1();
- inTupleDeserializer1 = new StreamRecordSerializer<IN1>(inTupleTypeInfo.createSerializer());
+ TypeInformation<IN1> inputTypeInfo1 = (TypeInformation<IN1>) typeWrapper.getInputTypeInfo1();
+ inputDeserializer1 = new StreamRecordSerializer<IN1>(inputTypeInfo1);
- inTupleTypeInfo = (TupleTypeInfo<IN1>) typeWrapper.getInputTupleTypeInfo2();
- inTupleDeserializer2 = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
+ TypeInformation<IN2> inputTypeInfo2 = (TypeInformation<IN2>) typeWrapper.getInputTypeInfo2();
+ inputDeserializer2 = new StreamRecordSerializer(inputTypeInfo2);
}
@Override
@@ -95,15 +95,15 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
setConfigOutputs(outputs);
setConfigInputs();
- inputIter1 = createInputIterator(inputs1, inTupleDeserializer1);
- inputIter2 = createInputIterator(inputs2, inTupleDeserializer2);
+ inputIter1 = createInputIterator(inputs1, inputDeserializer1);
+ inputIter2 = createInputIterator(inputs2, inputDeserializer2);
}
@Override
protected void setInvokable() {
userInvokable = getInvokable();
- userInvokable.initialize(collector, inputIter1, inTupleDeserializer1, inputIter2,
- inTupleDeserializer2, isMutable);
+ userInvokable.initialize(collector, inputIter1, inputDeserializer1, inputIter2,
+ inputDeserializer2, isMutable);
}
protected void setConfigInputs() throws StreamComponentException {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
index 8355b78..0c042bc 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
@@ -19,20 +19,19 @@
package org.apache.flink.streaming.api.streamcomponent;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.MutableRecordReader;
import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+import org.apache.flink.types.TypeInformation;
import org.apache.flink.util.MutableObjectIterator;
-public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT extends Tuple>
- extends AbstractStreamComponent<OUT> {
+public abstract class SingleInputAbstractStreamComponent<IN, OUT> extends
+ AbstractStreamComponent<OUT> {
- protected StreamRecordSerializer<IN> inTupleSerializer = null;
+ protected StreamRecordSerializer<IN> inputSerializer = null;
protected MutableObjectIterator<StreamRecord<IN>> inputIter;
protected MutableReader<IOReadableWritable> inputs;
@@ -46,17 +45,19 @@ public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT e
@SuppressWarnings("unchecked")
private void setDeserializer() {
- TupleTypeInfo<IN> inTupleTypeInfo = (TupleTypeInfo<IN>) typeWrapper
- .getInputTupleTypeInfo1();
- inTupleSerializer = new StreamRecordSerializer<IN>(inTupleTypeInfo.createSerializer());
+ TypeInformation<IN> inTupleTypeInfo = (TypeInformation<IN>) typeWrapper
+ .getInputTypeInfo1();
+ inputSerializer = new StreamRecordSerializer<IN>(inTupleTypeInfo);
}
@SuppressWarnings("unchecked")
protected void setSinkSerializer() {
- if (outSerializationDelegate != null) {
- TupleTypeInfo<IN> inTupleTypeInfo = (TupleTypeInfo<IN>) outTupleTypeInfo;
-
- inTupleSerializer = new StreamRecordSerializer<IN>(inTupleTypeInfo.createSerializer());
+ try {
+ TypeInformation<IN> inputTypeInfo = (TypeInformation<IN>) typeWrapper
+ .getOutputTypeInfo();
+ inputSerializer = new StreamRecordSerializer<IN>(inputTypeInfo);
+ } catch (RuntimeException e) {
+ // User implemented sink, nothing to do
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
index 911b550..436ebbf 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
@@ -48,7 +48,7 @@ public class StreamIterationSink<IN extends Tuple> extends
setConfigInputs();
setSinkSerializer();
- inputIter = createInputIterator(inputs, inTupleSerializer);
+ inputIter = createInputIterator(inputs, inputSerializer);
iterationId = configuration.getIterationId();
dataChannel = BlockingQueueBroker.instance().get(iterationId);
@@ -73,11 +73,11 @@ public class StreamIterationSink<IN extends Tuple> extends
}
protected void forwardRecords() throws Exception {
- StreamRecord<IN> reuse = inTupleSerializer.createInstance().setId(0);
+ StreamRecord<IN> reuse = inputSerializer.createInstance().setId(0);
while ((reuse = inputIter.next(reuse)) != null) {
pushToQueue(reuse);
// TODO: Fix object reuse for iteration
- reuse = inTupleSerializer.createInstance();
+ reuse = inputSerializer.createInstance();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
index 0ead3c6..2969e69 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
@@ -21,10 +21,9 @@ package org.apache.flink.streaming.api.streamcomponent;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
-public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamComponent<IN, IN> {
+public class StreamSink<IN> extends SingleInputAbstractStreamComponent<IN, IN> {
private static final Log LOG = LogFactory.getLog(StreamSink.class);
@@ -40,7 +39,7 @@ public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamCompo
setConfigInputs();
setSinkSerializer();
- inputIter = createInputIterator(inputs, inTupleSerializer);
+ inputIter = createInputIterator(inputs, inputSerializer);
} catch (Exception e) {
throw new StreamComponentException("Cannot register inputs for "
+ getClass().getSimpleName(), e);
@@ -50,7 +49,7 @@ public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamCompo
@Override
protected void setInvokable() {
userInvokable = getInvokable();
- userInvokable.initialize(collector, inputIter, inTupleSerializer, isMutable);
+ userInvokable.initialize(collector, inputIter, inputSerializer, isMutable);
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
index 7cb1d71..3a3f9cf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
@@ -51,13 +51,13 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
setConfigInputs();
setConfigOutputs(outputs);
- inputIter = createInputIterator(inputs, inTupleSerializer);
+ inputIter = createInputIterator(inputs, inputSerializer);
}
@Override
protected void setInvokable() {
userInvokable = getInvokable();
- userInvokable.initialize(collector, inputIter, inTupleSerializer, isMutable);
+ userInvokable.initialize(collector, inputIter, inputSerializer, isMutable);
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
index 9de2f38..bb21019 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
@@ -22,20 +22,21 @@ package org.apache.flink.streaming.api.streamrecord;
import java.io.Serializable;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
/**
- * Object for wrapping a tuple with ID used for sending records between
- * streaming task in Apache Flink stream processing.
+ * Object for wrapping a tuple or other object with ID used for sending records
+ * between streaming task in Apache Flink stream processing.
*/
-public class StreamRecord<T extends Tuple> implements Serializable {
+public class StreamRecord<T> implements Serializable {
private static final long serialVersionUID = 1L;
private UID uid;
- private T tuple;
-
- protected TupleSerializer<T> tupleSerializer;
+ private T streamObject;
+ public boolean isTuple;
+ /**
+ * Creates an empty StreamRecord and initializes an empty ID
+ */
public StreamRecord() {
uid = new UID();
}
@@ -60,28 +61,50 @@ public class StreamRecord<T extends Tuple> implements Serializable {
}
/**
+ * Gets the wrapped object from the StreamRecord
+ *
+ * @return The object wrapped
+ */
+ public T getObject() {
+ return streamObject;
+ }
+
+ /**
+ * Gets the field of the contained object at the given position. If a tuple
+ * is wrapped then the getField method is invoked. If the StreamRecord
+ * contains and object of Basic types only position 0 could be returned.
*
- * @return The tuple contained
+ * @param pos
+ * Position of the field to get.
+ * @return Returns the object contained in the position.
*/
- public T getTuple() {
- return tuple;
+ public Object getField(int pos) {
+ if (isTuple) {
+ return ((Tuple) streamObject).getField(pos);
+ } else {
+ if (pos == 0) {
+ return streamObject;
+ } else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
}
/**
- * Sets the tuple stored
+ * Sets the object stored
*
- * @param tuple
- * Value to set
+ * @param object
+ * Object to set
* @return Returns the StreamRecord object
*/
- public StreamRecord<T> setTuple(T tuple) {
- this.tuple = tuple;
+ public StreamRecord<T> setObject(T object) {
+ this.streamObject = object;
return this;
}
@Override
public String toString() {
- return tuple.toString();
+ return streamObject.toString();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
index 66c6751..66cb0bd 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
@@ -21,19 +21,20 @@ package org.apache.flink.streaming.api.streamrecord;
import java.io.IOException;
import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.types.TypeInformation;
-public final class StreamRecordSerializer<T extends Tuple> extends TypeSerializer<StreamRecord<T>> {
+public final class StreamRecordSerializer<T> extends TypeSerializer<StreamRecord<T>> {
private static final long serialVersionUID = 1L;
- private final TupleSerializer<T> tupleSerializer;
+ private final TypeSerializer<T> typeSerializer;
+ private final boolean isTuple;
- public StreamRecordSerializer(TupleSerializer<T> tupleSerializer) {
- this.tupleSerializer = tupleSerializer;
+ public StreamRecordSerializer(TypeInformation<T> typeInfo) {
+ this.typeSerializer = typeInfo.createSerializer();
+ this.isTuple = typeInfo.isTupleType();
}
@Override
@@ -51,7 +52,8 @@ public final class StreamRecordSerializer<T extends Tuple> extends TypeSerialize
try {
@SuppressWarnings("unchecked")
StreamRecord<T> t = StreamRecord.class.newInstance();
- t.setTuple(tupleSerializer.createInstance());
+ t.isTuple = isTuple;
+ t.setObject(typeSerializer.createInstance());
return t;
} catch (Exception e) {
throw new RuntimeException("Cannot instantiate StreamRecord.", e);
@@ -79,14 +81,14 @@ public final class StreamRecordSerializer<T extends Tuple> extends TypeSerialize
@Override
public void serialize(StreamRecord<T> value, DataOutputView target) throws IOException {
value.getId().write(target);
- tupleSerializer.serialize(value.getTuple(), target);
+ typeSerializer.serialize(value.getObject(), target);
}
@Override
public StreamRecord<T> deserialize(StreamRecord<T> reuse, DataInputView source)
throws IOException {
reuse.getId().read(source);
- tupleSerializer.deserialize(reuse.getTuple(), source);
+ reuse.setObject(typeSerializer.deserialize(reuse.getObject(), source));
return reuse;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java
index 06f059d..231bab1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.partitioner;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -29,7 +28,7 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T>
* Type of the Tuple
*/
-public class BroadcastPartitioner<T extends Tuple> implements StreamPartitioner<T> {
+public class BroadcastPartitioner<T> implements StreamPartitioner<T> {
private static final long serialVersionUID = 1L;
int[] returnArray;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/DistributePartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/DistributePartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/DistributePartitioner.java
index 025106a..9f492a3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/DistributePartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/DistributePartitioner.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.partitioner;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -30,7 +29,7 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T>
* Type of the Tuple
*/
-public class DistributePartitioner<T extends Tuple> implements StreamPartitioner<T> {
+public class DistributePartitioner<T> implements StreamPartitioner<T> {
private static final long serialVersionUID = 1L;
private int currentChannelIndex;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
index 51bb54e..44a674d 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.partitioner;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -30,7 +29,7 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T>
* Type of the Tuple
*/
-public class FieldsPartitioner<T extends Tuple> implements StreamPartitioner<T> {
+public class FieldsPartitioner<T> implements StreamPartitioner<T> {
private static final long serialVersionUID = 1L;
private int keyPosition;
@@ -44,7 +43,7 @@ public class FieldsPartitioner<T extends Tuple> implements StreamPartitioner<T>
@Override
public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
int numberOfOutputChannels) {
- returnArray[0] = Math.abs(record.getInstance().getTuple().getField(keyPosition).hashCode())
+ returnArray[0] = Math.abs(record.getInstance().getField(keyPosition).hashCode())
% numberOfOutputChannels;
return returnArray;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java
index 88da063..95ede97 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.partitioner;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -29,7 +28,7 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T>
* Type of the Tuple
*/
-public class ForwardPartitioner<T extends Tuple> implements StreamPartitioner<T> {
+public class ForwardPartitioner<T> implements StreamPartitioner<T> {
private static final long serialVersionUID = 1L;
private int[] returnArray;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java
index 649f4b7..83f7baf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java
@@ -19,12 +19,11 @@
package org.apache.flink.streaming.partitioner;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
//Group to the partitioner with the lowest id
-public class GlobalPartitioner<T extends Tuple> implements StreamPartitioner<T> {
+public class GlobalPartitioner<T> implements StreamPartitioner<T> {
private static final long serialVersionUID = 1L;
private int[] returnArray;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java
index 8d292e7..c56fecf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java
@@ -21,7 +21,6 @@ package org.apache.flink.streaming.partitioner;
import java.util.Random;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -32,7 +31,7 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T>
* Type of the Tuple
*/
-public class ShufflePartitioner<T extends Tuple> implements StreamPartitioner<T> {
+public class ShufflePartitioner<T> implements StreamPartitioner<T> {
private static final long serialVersionUID = 1L;
private Random random = new Random();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java
index c0e9c1a..18af2f6 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java
@@ -20,7 +20,6 @@ package org.apache.flink.streaming.partitioner;
import java.io.Serializable;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.io.network.api.ChannelSelector;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -31,6 +30,6 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T>
* Type of the Tuple
*/
-public interface StreamPartitioner<T extends Tuple> extends ChannelSelector<SerializationDelegate<StreamRecord<T>>>,
+public interface StreamPartitioner<T> extends ChannelSelector<SerializationDelegate<StreamRecord<T>>>,
Serializable {
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
index c868e4f..02a7554 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
@@ -22,11 +22,9 @@ package org.apache.flink.streaming.util.serialization;
import java.io.IOException;
import org.apache.flink.api.common.functions.AbstractFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
-public class FunctionTypeWrapper<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
+public class FunctionTypeWrapper<IN1, IN2, OUT> extends
TypeSerializerWrapper<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
@@ -44,30 +42,29 @@ public class FunctionTypeWrapper<IN1 extends Tuple, IN2 extends Tuple, OUT exten
this.inTypeParameter1 = inTypeParameter1;
this.inTypeParameter2 = inTypeParameter2;
this.outTypeParameter = outTypeParameter;
- setTupleTypeInfo();
+ setTypeInfo();
}
private void readObject(java.io.ObjectInputStream in) throws IOException,
ClassNotFoundException {
in.defaultReadObject();
- setTupleTypeInfo();
+ setTypeInfo();
}
- @SuppressWarnings({ "unchecked", "rawtypes" })
@Override
- protected void setTupleTypeInfo() {
+ protected void setTypeInfo() {
if (inTypeParameter1 != -1) {
- inTupleTypeInfo1 = (TupleTypeInfo) TypeExtractor.createTypeInfo(functionSuperClass,
+ inTypeInfo1 = TypeExtractor.createTypeInfo(functionSuperClass,
function.getClass(), inTypeParameter1, null, null);
}
if (inTypeParameter2 != -1) {
- inTupleTypeInfo2 = (TupleTypeInfo) TypeExtractor.createTypeInfo(functionSuperClass,
+ inTypeInfo2 = TypeExtractor.createTypeInfo(functionSuperClass,
function.getClass(), inTypeParameter2, null, null);
}
if (outTypeParameter != -1) {
- outTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(functionSuperClass,
+ outTypeInfo = TypeExtractor.createTypeInfo(functionSuperClass,
function.getClass(), outTypeParameter, null, null);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
index 5ed2312..c1bf52c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
@@ -21,41 +21,39 @@ package org.apache.flink.streaming.util.serialization;
import java.io.IOException;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
-public class ObjectTypeWrapper<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
+public class ObjectTypeWrapper<IN1, IN2, OUT> extends
TypeSerializerWrapper<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
- private Object inInstance1;
- private Object inInstance2;
- private Object outInstance;
+ private IN1 inInstance1;
+ private IN2 inInstance2;
+ private OUT outInstance;
- public ObjectTypeWrapper(Object inInstance1, Object inInstance2, Object outInstance) {
+ public ObjectTypeWrapper(IN1 inInstance1, IN2 inInstance2, OUT outInstance) {
this.inInstance1 = inInstance1;
this.inInstance2 = inInstance2;
this.outInstance = outInstance;
- setTupleTypeInfo();
+ setTypeInfo();
}
private void readObject(java.io.ObjectInputStream in) throws IOException,
ClassNotFoundException {
in.defaultReadObject();
- setTupleTypeInfo();
+ setTypeInfo();
}
@Override
- protected void setTupleTypeInfo() {
+ protected void setTypeInfo() {
if (inInstance1 != null) {
- inTupleTypeInfo1 = new TupleTypeInfo<IN1>(TypeExtractor.getForObject(inInstance1));
+ inTypeInfo1 = TypeExtractor.getForObject(inInstance1);
}
if (inInstance2 != null) {
- inTupleTypeInfo2 = new TupleTypeInfo<IN2>(TypeExtractor.getForObject(inInstance2));
+ inTypeInfo2 = TypeExtractor.getForObject(inInstance2);
}
if (outInstance != null) {
- outTupleTypeInfo = new TupleTypeInfo<OUT>(TypeExtractor.getForObject(outInstance));
+ outTypeInfo = TypeExtractor.getForObject(outInstance);
}
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
index 473ce7c..2aa50f2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
@@ -21,37 +21,36 @@ package org.apache.flink.streaming.util.serialization;
import java.io.Serializable;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.types.TypeInformation;
-public abstract class TypeSerializerWrapper<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple>
+public abstract class TypeSerializerWrapper<IN1, IN2, OUT>
implements Serializable {
private static final long serialVersionUID = 1L;
- protected transient TupleTypeInfo<IN1> inTupleTypeInfo1 = null;
- protected transient TupleTypeInfo<IN2> inTupleTypeInfo2 = null;
- protected transient TupleTypeInfo<OUT> outTupleTypeInfo = null;
+ protected transient TypeInformation<IN1> inTypeInfo1 = null;
+ protected transient TypeInformation<IN2> inTypeInfo2 = null;
+ protected transient TypeInformation<OUT> outTypeInfo = null;
- public TupleTypeInfo<IN1> getInputTupleTypeInfo1() {
- if (inTupleTypeInfo1 == null) {
+ public TypeInformation<IN1> getInputTypeInfo1() {
+ if (inTypeInfo1 == null) {
throw new RuntimeException("There is no TypeInfo for the first input");
}
- return inTupleTypeInfo1;
+ return inTypeInfo1;
}
- public TupleTypeInfo<IN2> getInputTupleTypeInfo2() {
- if (inTupleTypeInfo1 == null) {
- throw new RuntimeException("There is no TypeInfo for the first input");
+ public TypeInformation<IN2> getInputTypeInfo2() {
+ if (inTypeInfo2 == null) {
+ throw new RuntimeException("There is no TypeInfo for the second input");
}
- return inTupleTypeInfo2;
+ return inTypeInfo2;
}
- public TupleTypeInfo<OUT> getOutputTupleTypeInfo() {
- if (inTupleTypeInfo1 == null) {
- throw new RuntimeException("There is no TypeInfo for the first input");
+ public TypeInformation<OUT> getOutputTypeInfo() {
+ if (outTypeInfo == null) {
+ throw new RuntimeException("There is no TypeInfo for the output");
}
- return outTupleTypeInfo;
+ return outTypeInfo;
}
- protected abstract void setTupleTypeInfo();
+ protected abstract void setTypeInfo();
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 8ba58c5..1ea165f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -25,7 +25,6 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.flink.util.Collector;
@@ -38,13 +37,13 @@ public class IterateTest {
private static boolean iterated = false;
public static final class IterationHead extends
- FlatMapFunction<Tuple1<Boolean>, Tuple1<Boolean>> {
+ FlatMapFunction<Boolean, Boolean> {
private static final long serialVersionUID = 1L;
@Override
- public void flatMap(Tuple1<Boolean> value, Collector<Tuple1<Boolean>> out) throws Exception {
- if (value.f0) {
+ public void flatMap(Boolean value, Collector<Boolean> out) throws Exception {
+ if (value) {
iterated = true;
} else {
out.collect(value);
@@ -55,24 +54,24 @@ public class IterateTest {
}
public static final class IterationTail extends
- FlatMapFunction<Tuple1<Boolean>, Tuple1<Boolean>> {
+ FlatMapFunction<Boolean,Boolean> {
private static final long serialVersionUID = 1L;
@Override
- public void flatMap(Tuple1<Boolean> value, Collector<Tuple1<Boolean>> out) throws Exception {
- out.collect(new Tuple1<Boolean>(true));
+ public void flatMap(Boolean value, Collector<Boolean> out) throws Exception {
+ out.collect(true);
}
}
- public static final class MySink extends SinkFunction<Tuple1<Boolean>> {
+ public static final class MySink extends SinkFunction<Boolean> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Boolean> tuple) {
+ public void invoke(Boolean tuple) {
}
}
@@ -87,12 +86,12 @@ public class IterateTest {
for (int i = 0; i < 100000; i++) {
bl.add(false);
}
- DataStream<Tuple1<Boolean>> source = env
+ DataStream<Boolean> source = env
.fromCollection(bl);
- IterativeDataStream<Tuple1<Boolean>> iteration = source.iterate();
+ IterativeDataStream<Boolean> iteration = source.iterate();
- DataStream<Tuple1<Boolean>> increment = iteration.flatMap(new IterationHead()).flatMap(new IterationTail());
+ DataStream<Boolean> increment = iteration.flatMap(new IterationHead()).flatMap(new IterationTail());
iteration.closeWith(increment).addSink(new MySink());
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
index 67dce9d..438887a 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
@@ -35,7 +35,7 @@ public class PrintTest {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
- env.generateSequence(1, 10).print();
+ env.generateSequence(1, 100000).print();
env.executeTest(MEMORYSIZE);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index 8f64cb4..e8742de 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -26,7 +26,6 @@ import java.util.Collection;
import java.util.HashSet;
import org.apache.flink.api.java.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
@@ -39,45 +38,45 @@ public class DirectedOutputTest {
static HashSet<Long> evenSet = new HashSet<Long>();
static HashSet<Long> oddSet = new HashSet<Long>();
- private static class PlusTwo extends MapFunction<Tuple1<Long>, Tuple1<Long>> {
+ private static class PlusTwo extends MapFunction<Long, Long> {
private static final long serialVersionUID = 1L;
@Override
- public Tuple1<Long> map(Tuple1<Long> arg0) throws Exception {
- arg0.f0 += 2;
+ public Long map(Long arg0) throws Exception {
+ arg0 += 2;
return arg0;
}
}
- private static class EvenSink extends SinkFunction<Tuple1<Long>> {
+ private static class EvenSink extends SinkFunction<Long> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Long> tuple) {
- evenSet.add(tuple.f0);
+ public void invoke(Long tuple) {
+ evenSet.add(tuple);
}
}
- private static class OddSink extends SinkFunction<Tuple1<Long>> {
+ private static class OddSink extends SinkFunction<Long> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Long> tuple) {
- oddSet.add(tuple.f0);
+ public void invoke(Long tuple) {
+ oddSet.add(tuple);
}
}
- private static class MySelector extends OutputSelector<Tuple1<Long>> {
+ private static class MySelector extends OutputSelector<Long> {
private static final long serialVersionUID = 1L;
@Override
- public void select(Tuple1<Long> tuple, Collection<String> outputs) {
- int mod = (int) (tuple.f0 % 2);
+ public void select(Long tuple, Collection<String> outputs) {
+ int mod = (int) (tuple % 2);
switch (mod) {
case 0:
outputs.add("ds1");
@@ -96,10 +95,10 @@ public class DirectedOutputTest {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
- DataStream<Tuple1<Long>> s = env.generateSequence(1, 6).directTo(new MySelector());
- DataStream<Tuple1<Long>> ds1 = s.map(new PlusTwo()).name("ds1").addSink(new EvenSink());
- DataStream<Tuple1<Long>> ds2 = s.map(new PlusTwo()).name("ds2").addSink(new OddSink());
- DataStream<Tuple1<Long>> ds3 = s.map(new PlusTwo()).addSink(new OddSink());
+ DataStream<Long> s = env.generateSequence(1, 6).directTo(new MySelector());
+ DataStream<Long> ds1 = s.map(new PlusTwo()).name("ds1").addSink(new EvenSink());
+ DataStream<Long> ds2 = s.map(new PlusTwo()).name("ds2").addSink(new OddSink());
+ DataStream<Long> ds3 = s.map(new PlusTwo()).addSink(new OddSink());
env.execute();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
index d3cbcbc..2c3f480 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
@@ -23,7 +23,6 @@ import java.io.Serializable;
import java.util.HashSet;
import java.util.Set;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
@@ -40,28 +39,28 @@ public class CoMapTest implements Serializable {
private static Set<String> result;
private static Set<String> expected = new HashSet<String>();
- private final static class EmptySink extends SinkFunction<Tuple1<Boolean>> {
+ private final static class EmptySink extends SinkFunction<Boolean> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Boolean> tuple) {
+ public void invoke(Boolean tuple) {
}
}
private final static class MyCoMap extends
- CoMapFunction<Tuple1<String>, Tuple1<Integer>, Tuple1<Boolean>> {
+ CoMapFunction<String, Integer, Boolean> {
private static final long serialVersionUID = 1L;
@Override
- public Tuple1<Boolean> map1(Tuple1<String> value) {
- result.add(value.f0);
- return new Tuple1<Boolean>(true);
+ public Boolean map1(String value) {
+ result.add(value);
+ return true;
}
@Override
- public Tuple1<Boolean> map2(Tuple1<Integer> value) {
- result.add(value.f0.toString());
- return new Tuple1<Boolean>(false);
+ public Boolean map2(Integer value) {
+ result.add(value.toString());
+ return false;
}
}
@@ -80,14 +79,14 @@ public class CoMapTest implements Serializable {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
- DataStream<Tuple1<Integer>> ds1 = env.fromElements(1, 3);
+ DataStream<Integer> ds1 = env.fromElements(1, 3);
@SuppressWarnings("unchecked")
- DataStream<Tuple1<Integer>> ds2 = env.fromElements(2, 4).connectWith(ds1);
+ DataStream<Integer> ds2 = env.fromElements(2, 4).connectWith(ds1);
- DataStream<Tuple1<String>> ds3 = env.fromElements("a", "b");
+ DataStream<String> ds3 = env.fromElements("a", "b");
@SuppressWarnings({ "unused", "unchecked" })
- DataStream<Tuple1<Boolean>> ds4 = env.fromElements("c").connectWith(ds3)
+ DataStream<Boolean> ds4 = env.fromElements("c").connectWith(ds3)
.coMapWith(new MyCoMap(),
ds2).addSink(new EmptySink());
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
index 054becc..3286ef1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
@@ -24,7 +24,6 @@ import java.util.HashSet;
import java.util.Set;
import org.apache.flink.api.java.functions.FilterFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
@@ -34,37 +33,37 @@ import org.junit.Test;
public class FilterTest implements Serializable {
private static final long serialVersionUID = 1L;
-
+
private static Set<Integer> set = new HashSet<Integer>();
- private static class SetSink extends SinkFunction<Tuple1<Integer>> {
+ private static class MySink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
- set.add(tuple.f0);
+ public void invoke(Integer value) {
+ set.add(value);
}
}
- static class MyFilter extends FilterFunction<Tuple1<Integer>> {
+ static class MyFilter extends FilterFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public boolean filter(Tuple1<Integer> value) throws Exception {
- return value.f0 % 2 == 0;
+ public boolean filter(Integer value) throws Exception {
+ return value % 2 == 0;
}
}
@Test
public void test() {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
-
+
StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
- env.fromElements(1, 2, 3, 4, 5, 6, 7).filter(new MyFilter()).addSink(new SetSink());
+
+ env.fromElements(1, 2, 3, 4, 5, 6, 7).filter(new MyFilter()).addSink(new MySink());
env.execute();
-
+
Assert.assertArrayEquals(new Integer[] { 2, 4, 6 }, set.toArray());
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
index 893ec37..b299407 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
@@ -25,91 +25,88 @@ import static org.junit.Assert.assertTrue;
import java.util.HashSet;
import java.util.Set;
+import org.apache.flink.api.java.functions.FlatMapFunction;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
-import org.junit.Test;
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
import org.apache.log4j.Level;
+import org.junit.Test;
public class FlatMapTest {
- public static final class MyFlatMap extends FlatMapFunction<Tuple1<Integer>, Tuple1<Integer>> {
+ public static final class MyFlatMap extends FlatMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void flatMap(Tuple1<Integer> value, Collector<Tuple1<Integer>> out) throws Exception {
- out.collect(new Tuple1<Integer>(value.f0 * value.f0));
+ public void flatMap(Integer value, Collector<Integer> out) throws Exception {
+ out.collect(value * value);
}
}
- public static final class ParallelFlatMap extends
- FlatMapFunction<Tuple1<Integer>, Tuple1<Integer>> {
+ public static final class ParallelFlatMap extends FlatMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void flatMap(Tuple1<Integer> value, Collector<Tuple1<Integer>> out) throws Exception {
+ public void flatMap(Integer value, Collector<Integer> out) throws Exception {
numberOfElements++;
}
}
- public static final class GenerateSequenceFlatMap extends
- FlatMapFunction<Tuple1<Long>, Tuple1<Long>> {
+ public static final class GenerateSequenceFlatMap extends FlatMapFunction<Long, Long> {
private static final long serialVersionUID = 1L;
@Override
- public void flatMap(Tuple1<Long> value, Collector<Tuple1<Long>> out) throws Exception {
- out.collect(new Tuple1<Long>(value.f0 * value.f0));
+ public void flatMap(Long value, Collector<Long> out) throws Exception {
+ out.collect(value * value);
}
}
- public static final class MySink extends SinkFunction<Tuple1<Integer>> {
+ public static final class MySink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
- result.add(tuple.f0);
+ public void invoke(Integer tuple) {
+ result.add(tuple);
}
}
- public static final class FromElementsSink extends SinkFunction<Tuple1<Integer>> {
+ public static final class FromElementsSink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
- fromElementsResult.add(tuple.f0);
+ public void invoke(Integer tuple) {
+ fromElementsResult.add(tuple);
}
}
- public static final class FromCollectionSink extends SinkFunction<Tuple1<Integer>> {
+ public static final class FromCollectionSink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
- fromCollectionResult.add(tuple.f0);
+ public void invoke(Integer tuple) {
+ fromCollectionResult.add(tuple);
}
}
- public static final class GenerateSequenceSink extends SinkFunction<Tuple1<Long>> {
+ public static final class GenerateSequenceSink extends SinkFunction<Long> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Long> tuple) {
- generateSequenceResult.add(tuple.f0);
+ public void invoke(Long tuple) {
+ generateSequenceResult.add(tuple);
}
}
@@ -170,7 +167,7 @@ public class FlatMapTest {
fillFromCollectionSet();
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream = env.fromCollection(fromCollectionSet)
+ DataStream<Integer> dataStream = env.fromCollection(fromCollectionSet)
.flatMap(new MyFlatMap()).addSink(new MySink());
fillExpectedList();
@@ -178,43 +175,38 @@ public class FlatMapTest {
// parallelShuffleconnectTest
fillFromCollectionSet();
- DataStream<Tuple1<Integer>> source = env.fromCollection(fromCollectionSet);
+ DataStream<Integer> source = env.fromCollection(fromCollectionSet);
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> map = source
- .flatMap(new ParallelFlatMap())
- .addSink(new MySink());
+ DataStream<Integer> map = source.flatMap(new ParallelFlatMap()).addSink(
+ new MySink());
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> map2 = source
- .flatMap(new ParallelFlatMap())
- .addSink(new MySink());
+ DataStream<Integer> map2 = source.flatMap(new ParallelFlatMap()).addSink(
+ new MySink());
// fromElementsTest
- DataStream<Tuple1<Integer>> fromElementsMap = env
- .fromElements(2, 5, 9)
- .flatMap(new MyFlatMap());
+ DataStream<Integer> fromElementsMap = env.fromElements(2, 5, 9).flatMap(
+ new MyFlatMap());
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> sink = fromElementsMap.addSink(new FromElementsSink());
+ DataStream<Integer> sink = fromElementsMap.addSink(new FromElementsSink());
fillFromElementsExpected();
// fromCollectionTest
fillFromCollectionSet();
- DataStream<Tuple1<Integer>> fromCollectionMap = env
- .fromCollection(fromCollectionSet)
+ DataStream<Integer> fromCollectionMap = env.fromCollection(fromCollectionSet)
.flatMap(new MyFlatMap());
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> fromCollectionSink = fromCollectionMap
+ DataStream<Integer> fromCollectionSink = fromCollectionMap
.addSink(new FromCollectionSink());
// generateSequenceTest
fillSequenceSet();
- DataStream<Tuple1<Long>> generateSequenceMap = env
- .generateSequence(0, 9)
- .flatMap(new GenerateSequenceFlatMap());
+ DataStream<Long> generateSequenceMap = env.generateSequence(0, 9).flatMap(
+ new GenerateSequenceFlatMap());
@SuppressWarnings("unused")
- DataStream<Tuple1<Long>> generateSequenceSink = generateSequenceMap
+ DataStream<Long> generateSequenceSink = generateSequenceMap
.addSink(new GenerateSequenceSink());
fillLongSequenceSet();
[33/51] [abbrv] git commit: [streaming] Updated Streaming function
interfaces to match main project
Posted by se...@apache.org.
[streaming] Updated Streaming function interfaces to match main project
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/e73ea295
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/e73ea295
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/e73ea295
Branch: refs/heads/master
Commit: e73ea29593cdb8b4d9a11137b2188ca72673e98c
Parents: 0465d30
Author: gyfora <gy...@gmail.com>
Authored: Mon Aug 4 14:05:07 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:50 2014 +0200
----------------------------------------------------------------------
.../streaming/connectors/flume/FlumeSink.java | 12 +-
.../streaming/connectors/flume/FlumeSource.java | 4 +-
.../streaming/connectors/kafka/KafkaSink.java | 12 +-
.../streaming/connectors/kafka/KafkaSource.java | 4 +-
.../connectors/kafka/KafkaTopology.java | 2 +-
.../streaming/connectors/rabbitmq/RMQSink.java | 12 +-
.../connectors/rabbitmq/RMQSource.java | 4 +-
.../connectors/twitter/TwitterSource.java | 2 +-
.../connectors/twitter/TwitterStreaming.java | 33 +++---
.../streaming/connectors/rabbitmq/RMQTest.java | 46 ++++----
.../apache/flink/streaming/api/DataStream.java | 110 +++++++++++--------
.../streaming/api/collector/OutputSelector.java | 4 +-
.../api/function/co/CoMapFunction.java | 13 ++-
.../api/function/co/RichCoMapFunction.java | 27 +++++
.../api/function/sink/PrintSinkFunction.java | 2 +-
.../api/function/sink/RichSinkFunction.java | 30 +++++
.../api/function/sink/SinkFunction.java | 6 +-
.../api/function/sink/WriteSinkFunction.java | 2 +-
.../api/function/source/FileSourceFunction.java | 2 +-
.../api/function/source/FileStreamFunction.java | 2 +-
.../function/source/FromElementsFunction.java | 2 +-
.../function/source/GenSequenceFunction.java | 2 +-
.../api/function/source/RichSourceFunction.java | 29 +++++
.../api/function/source/SourceFunction.java | 10 +-
.../streaming/api/invokable/SinkInvokable.java | 9 +-
.../api/invokable/SourceInvokable.java | 9 +-
.../operator/BatchReduceInvokable.java | 4 +-
.../api/invokable/operator/FilterInvokable.java | 15 ++-
.../invokable/operator/FlatMapInvokable.java | 15 ++-
.../api/invokable/operator/MapInvokable.java | 15 ++-
.../operator/StreamReduceInvokable.java | 13 ++-
.../operator/WindowReduceInvokable.java | 4 +-
.../invokable/operator/co/CoMapInvokable.java | 9 +-
.../api/streamcomponent/CoStreamTask.java | 4 +-
.../util/serialization/FunctionTypeWrapper.java | 26 ++---
.../apache/flink/streaming/api/IterateTest.java | 2 +-
.../flink/streaming/api/WriteAsCsvTest.java | 2 +-
.../flink/streaming/api/WriteAsTextTest.java | 2 +-
.../api/collector/DirectedOutputTest.java | 4 +-
.../api/invokable/operator/BatchReduceTest.java | 10 +-
.../api/invokable/operator/CoMapTest.java | 4 +-
.../api/invokable/operator/FilterTest.java | 6 +-
.../api/invokable/operator/FlatMapTest.java | 16 +--
.../api/invokable/operator/MapTest.java | 36 +++---
.../streamcomponent/StreamComponentTest.java | 4 +-
.../examples/basictopology/BasicTopology.java | 6 +-
.../examples/cellinfo/CellInfoLocal.java | 4 +-
.../CollaborativeFilteringSink.java | 2 +-
.../CollaborativeFilteringSource.java | 2 +-
.../examples/iterative/kmeans/KMeansSink.java | 2 +-
.../examples/iterative/kmeans/KMeansSource.java | 2 +-
.../iterative/pagerank/PageRankSink.java | 2 +-
.../iterative/pagerank/PageRankSource.java | 2 +-
.../examples/iterative/sssp/SSSPSink.java | 2 +-
.../examples/iterative/sssp/SSSPSource.java | 2 +-
.../flink/streaming/examples/join/JoinSink.java | 2 +-
.../streaming/examples/join/JoinSourceOne.java | 2 +-
.../streaming/examples/join/JoinSourceTwo.java | 2 +-
.../ml/IncrementalLearningSkeleton.java | 4 +-
.../streaming/examples/ml/IncrementalOLS.java | 4 +-
.../window/join/WindowJoinSourceOne.java | 2 +-
.../window/join/WindowJoinSourceTwo.java | 2 +-
.../examples/wordcount/WordCountCounter.java | 6 +-
.../examples/wordcount/WordCountSplitter.java | 6 +-
.../testdata_checksum/ASTopology.data.md5 | 1 +
.../testdata_checksum/MovieLens100k.data.md5 | 1 +
.../resources/testdata_checksum/hamlet.txt.md5 | 1 +
.../testdata_checksum/terainput.txt.md5 | 1 +
68 files changed, 389 insertions(+), 252 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
index 6f943d1..69e34e6 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
@@ -30,7 +30,7 @@ import org.apache.flume.api.RpcClient;
import org.apache.flume.api.RpcClientFactory;
import org.apache.flume.event.EventBuilder;
-public abstract class FlumeSink<IN> extends SinkFunction<IN> {
+public abstract class FlumeSink<IN> implements SinkFunction<IN> {
private static final long serialVersionUID = 1L;
private static final Log LOG = LogFactory.getLog(FlumeSink.class);
@@ -51,18 +51,18 @@ public abstract class FlumeSink<IN> extends SinkFunction<IN> {
* Receives tuples from the Apache Flink {@link DataStream} and forwards them to
* Apache Flume.
*
- * @param tuple
+ * @param value
* The tuple arriving from the datastream
*/
@Override
- public void invoke(IN tuple) {
+ public void invoke(IN value) {
if (!initDone) {
client = new FlinkRpcClientFacade();
client.init(host, port);
}
- byte[] data = serialize(tuple);
+ byte[] data = serialize(value);
if (!closeWithoutSend) {
client.sendDataToFlume(data);
}
@@ -75,11 +75,11 @@ public abstract class FlumeSink<IN> extends SinkFunction<IN> {
/**
* Serializes tuples into byte arrays.
*
- * @param tuple
+ * @param value
* The tuple used for the serialization
* @return The serialized byte array.
*/
- public abstract byte[] serialize(IN tuple);
+ public abstract byte[] serialize(IN value);
private class FlinkRpcClientFacade {
private RpcClient client;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
index b141efb..c296319 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
@@ -22,7 +22,7 @@ package org.apache.flink.streaming.connectors.flume;
import java.util.List;
import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.streaming.api.function.source.RichSourceFunction;
import org.apache.flink.util.Collector;
import org.apache.flume.Context;
import org.apache.flume.channel.ChannelProcessor;
@@ -30,7 +30,7 @@ import org.apache.flume.source.AvroSource;
import org.apache.flume.source.avro.AvroFlumeEvent;
import org.apache.flume.source.avro.Status;
-public abstract class FlumeSource<OUT> extends SourceFunction<OUT> {
+public abstract class FlumeSource<OUT> extends RichSourceFunction<OUT> {
private static final long serialVersionUID = 1L;
String host;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
index 955e8dc..183860e 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
@@ -27,7 +27,7 @@ import kafka.producer.ProducerConfig;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
-public abstract class KafkaSink<IN, OUT> extends SinkFunction<IN> {
+public abstract class KafkaSink<IN, OUT> implements SinkFunction<IN> {
private static final long serialVersionUID = 1L;
private kafka.javaapi.producer.Producer<Integer, OUT> producer;
@@ -62,16 +62,16 @@ public abstract class KafkaSink<IN, OUT> extends SinkFunction<IN> {
/**
* Called when new data arrives to the sink, and forwards it to Kafka.
*
- * @param tuple
+ * @param value
* The incoming data
*/
@Override
- public void invoke(IN tuple) {
+ public void invoke(IN value) {
if (!initDone) {
initialize();
}
- OUT out = serialize(tuple);
+ OUT out = serialize(value);
KeyedMessage<Integer, OUT> data = new KeyedMessage<Integer, OUT>(topicId, out);
if (!closeWithoutSend) {
@@ -86,11 +86,11 @@ public abstract class KafkaSink<IN, OUT> extends SinkFunction<IN> {
/**
* Serializes tuples into byte arrays.
*
- * @param tuple
+ * @param value
* The tuple used for the serialization
* @return The serialized byte array.
*/
- public abstract OUT serialize(IN tuple);
+ public abstract OUT serialize(IN value);
/**
* Closes the connection immediately and no further data will be sent.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
index 228069a..6e18b20 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
@@ -30,10 +30,10 @@ import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.streaming.api.function.source.RichSourceFunction;
import org.apache.flink.util.Collector;
-public abstract class KafkaSource<OUT> extends SourceFunction<OUT> {
+public abstract class KafkaSource<OUT> extends RichSourceFunction<OUT> {
private static final long serialVersionUID = 1L;
private final String zkQuorum;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
index d605fb8..295f1cc 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
@@ -27,7 +27,7 @@ import org.apache.flink.util.Collector;
public class KafkaTopology {
- public static final class MySource extends SourceFunction<Tuple1<String>> {
+ public static final class MySource implements SourceFunction<Tuple1<String>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index c6f0ef5..2e3a8a6 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -29,7 +29,7 @@ import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
import com.rabbitmq.client.ConnectionFactory;
-public abstract class RMQSink<IN> extends SinkFunction<IN> {
+public abstract class RMQSink<IN> implements SinkFunction<IN> {
private static final long serialVersionUID = 1L;
private static final Log LOG = LogFactory.getLog(RMQSink.class);
@@ -69,18 +69,18 @@ public abstract class RMQSink<IN> extends SinkFunction<IN> {
/**
* Called when new data arrives to the sink, and forwards it to RMQ.
*
- * @param tuple
+ * @param value
* The incoming data
*/
@Override
- public void invoke(IN tuple) {
+ public void invoke(IN value) {
if (!initDone) {
initializeConnection();
}
try {
channel.queueDeclare(QUEUE_NAME, false, false, false, null);
- byte[] msg = serialize(tuple);
+ byte[] msg = serialize(value);
if (!closeWithoutSend) {
channel.basicPublish("", QUEUE_NAME, null, msg);
}
@@ -98,11 +98,11 @@ public abstract class RMQSink<IN> extends SinkFunction<IN> {
/**
* Serializes tuples into byte arrays.
*
- * @param tuple
+ * @param value
* The tuple used for the serialization
* @return The serialized byte array.
*/
- public abstract byte[] serialize(IN tuple);
+ public abstract byte[] serialize(IN value);
/**
* Closes the connection.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index 8303b1a..fa0be0d 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -24,7 +24,7 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.streaming.api.function.source.RichSourceFunction;
import org.apache.flink.util.Collector;
import com.rabbitmq.client.Channel;
@@ -32,7 +32,7 @@ import com.rabbitmq.client.Connection;
import com.rabbitmq.client.ConnectionFactory;
import com.rabbitmq.client.QueueingConsumer;
-public abstract class RMQSource<OUT> extends SourceFunction<OUT> {
+public abstract class RMQSource<OUT> extends RichSourceFunction<OUT> {
private static final long serialVersionUID = 1L;
private static final Log LOG = LogFactory.getLog(RMQSource.class);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
index bc0995d..17e3b02 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
@@ -44,7 +44,7 @@ import com.twitter.hbc.httpclient.auth.OAuth1;
* Implementation of {@link SourceFunction} specialized to emit tweets from Twitter.
* It can connect to Twitter Streaming API, collect tweets and
*/
-public class TwitterSource extends SourceFunction<String> {
+public class TwitterSource implements SourceFunction<String> {
private static final Log LOG = LogFactory.getLog(TwitterSource.class);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
index ee986ea..6a464ea 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
@@ -31,7 +31,8 @@ public class TwitterStreaming {
private static final int PARALLELISM = 1;
private static final int SOURCE_PARALLELISM = 1;
- public static class TwitterSink extends SinkFunction<Tuple5<Long, Long, String, String, String>> {
+ public static class TwitterSink implements
+ SinkFunction<Tuple5<Long, Long, String, String, String>> {
private static final long serialVersionUID = 1L;
@@ -44,40 +45,38 @@ public class TwitterStreaming {
}
}
-
+
public static class SelectDataFlatMap extends
JSONParseFlatMap<String, Tuple5<Long, Long, String, String, String>> {
private static final long serialVersionUID = 1L;
@Override
- public void flatMap(String value,
- Collector<Tuple5<Long, Long, String, String, String>> out)
+ public void flatMap(String value, Collector<Tuple5<Long, Long, String, String, String>> out)
throws Exception {
out.collect(new Tuple5<Long, Long, String, String, String>(
convertDateString2Long(getField(value, "id")),
convertDateString2LongDate(getField(value, "created_at")),
- colationOfNull(getField(value, "user.name")),
- colationOfNull(getField(value, "text")),
- getField(value, "lang")));
+ colationOfNull(getField(value, "user.name")), colationOfNull(getField(value,
+ "text")), getField(value, "lang")));
}
-
- protected String colationOfNull(String in){
- if(in==null){
+
+ protected String colationOfNull(String in) {
+ if (in == null) {
return " ";
}
return in;
}
-
+
protected Long convertDateString2LongDate(String dateString) {
- if (dateString!=(null)) {
+ if (dateString != (null)) {
String[] dateArray = dateString.split(" ");
- return Long.parseLong(dateArray[2])*100000+Long.parseLong(dateArray[5]);
+ return Long.parseLong(dateArray[2]) * 100000 + Long.parseLong(dateArray[5]);
}
return 0L;
}
-
+
protected Long convertDateString2Long(String dateString) {
if (dateString != null) {
return Long.parseLong(dateString);
@@ -87,14 +86,14 @@ public class TwitterStreaming {
}
public static void main(String[] args) {
-
+
String path = "/home/eszes/git/auth.properties";
StreamExecutionEnvironment env = StreamExecutionEnvironment
.createLocalEnvironment(PARALLELISM);
- DataStream<String> streamSource = env.addSource(
- new TwitterSource(path,100), SOURCE_PARALLELISM);
+ DataStream<String> streamSource = env.addSource(new TwitterSource(path, 100),
+ SOURCE_PARALLELISM);
DataStream<Tuple5<Long, Long, String, String, String>> selectedDataStream = streamSource
.flatMap(new SelectDataFlatMap());
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java
index c6a43f2..ad704d7 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.connectors.rabbitmq;
-
import java.util.HashSet;
import java.util.Set;
@@ -28,8 +27,8 @@ import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.junit.Test;
public class RMQTest {
-
- public static final class MySink extends SinkFunction<Tuple1<String>> {
+
+ public static final class MySink implements SinkFunction<Tuple1<String>> {
private static final long serialVersionUID = 1L;
@Override
@@ -37,38 +36,37 @@ public class RMQTest {
result.add(tuple.f0);
}
-
}
-
+
private static Set<String> expected = new HashSet<String>();
private static Set<String> result = new HashSet<String>();
-
+
@SuppressWarnings("unused")
- private static void fillExpected() {
+ private static void fillExpected() {
expected.add("one");
expected.add("two");
expected.add("three");
expected.add("four");
expected.add("five");
}
-
+
@Test
public void RMQTest1() throws Exception {
-//
-// StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-//
-// DataStream<Tuple1<String>> dataStream1 = env
-// .addSource(new RMQSource("localhost", "hello"), 1)
-// .addSink(new MySink());
-//
-// DataStream<Tuple1<String>> dataStream2 = env
-// .fromElements("one", "two", "three", "four", "five", "q")
-// .addSink(new RMQSink("localhost", "hello"));
-//
-// env.execute();
-//
-// fillExpected();
-//
-// assertEquals(expected, result);
+ //
+ // StreamExecutionEnvironment env = new StreamExecutionEnvironment();
+ //
+ // DataStream<Tuple1<String>> dataStream1 = env
+ // .addSource(new RMQSource("localhost", "hello"), 1)
+ // .addSink(new MySink());
+ //
+ // DataStream<Tuple1<String>> dataStream2 = env
+ // .fromElements("one", "two", "three", "four", "five", "q")
+ // .addSink(new RMQSink("localhost", "hello"));
+ //
+ // env.execute();
+ //
+ // fillExpected();
+ //
+ // assertEquals(expected, result);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index 430f09b..7aff259 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -19,19 +19,24 @@
package org.apache.flink.streaming.api;
+import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.lang3.SerializationException;
import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.java.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.api.java.functions.RichGroupReduceFunction;
import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
+import org.apache.flink.streaming.api.function.co.RichCoMapFunction;
import org.apache.flink.streaming.api.function.sink.PrintSinkFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
@@ -184,7 +189,8 @@ public class DataStream<T> {
* Sets the mutability of the operator represented by the DataStream. If the
* operator is set to mutable, the tuples received in the user defined
* functions, will be reused after the function call. Setting an operator to
- * mutable greatly reduces garbage collection overhead and thus scalability.
+ * mutable reduces garbage collection overhead and thus increases
+ * scalability.
*
* @param isMutable
* The mutability of the operator.
@@ -309,38 +315,42 @@ public class DataStream<T> {
/**
* Applies a Map transformation on a {@link DataStream}. The transformation
- * calls a {@link RichMapFunction} for each element of the DataStream. Each
- * MapFunction call returns exactly one element.
+ * calls a {@link MapFunction} for each element of the DataStream. Each
+ * MapFunction call returns exactly one element. The user can also extend
+ * {@link RichMapFunction} to gain access to other features provided by the
+ * {@link RichFuntion} interface.
*
* @param mapper
- * The RichMapFunction that is called for each element of the
+ * The MapFunction that is called for each element of the
* DataStream.
* @param <R>
* output type
* @return The transformed DataStream.
*/
- public <R> StreamOperator<R> map(RichMapFunction<T, R> mapper) {
+ public <R> StreamOperator<R> map(MapFunction<T, R> mapper) {
return addFunction("map", mapper, new FunctionTypeWrapper<T, Tuple, R>(mapper,
- RichMapFunction.class, 0, -1, 1), new MapInvokable<T, R>(mapper));
+ MapFunction.class, 0, -1, 1), new MapInvokable<T, R>(mapper));
}
/**
* Applies a FlatMap transformation on a {@link DataStream}. The
- * transformation calls a {@link RichFlatMapFunction} for each element of
- * the DataStream. Each RichFlatMapFunction call can return any number of
- * elements including none.
+ * transformation calls a {@link FlatMapFunction} for each element of the
+ * DataStream. Each FlatMapFunction call can return any number of elements
+ * including none. The user can also extend {@link RichFlatMapFunction} to
+ * gain access to other features provided by the {@link RichFuntion}
+ * interface.
*
* @param flatMapper
- * The RichFlatMapFunction that is called for each element of the
+ * The FlatMapFunction that is called for each element of the
* DataStream
*
* @param <R>
* output type
* @return The transformed DataStream.
*/
- public <R> StreamOperator<R> flatMap(RichFlatMapFunction<T, R> flatMapper) {
+ public <R> StreamOperator<R> flatMap(FlatMapFunction<T, R> flatMapper) {
return addFunction("flatMap", flatMapper, new FunctionTypeWrapper<T, Tuple, R>(flatMapper,
- RichFlatMapFunction.class, 0, -1, 1), new FlatMapInvokable<T, R>(flatMapper));
+ FlatMapFunction.class, 0, -1, 1), new FlatMapInvokable<T, R>(flatMapper));
}
/**
@@ -348,7 +358,9 @@ public class DataStream<T> {
* transformation calls a {@link CoMapFunction#map1(Tuple)} for each element
* of the first DataStream (on which .coMapWith was called) and
* {@link CoMapFunction#map2(Tuple)} for each element of the second
- * DataStream. Each CoMapFunction call returns exactly one element.
+ * DataStream. Each CoMapFunction call returns exactly one element. The user
+ * can also extend {@link RichCoMapFunction} to gain access to other
+ * features provided by the {@link RichFuntion} interface.
*
* @param coMapper
* The CoMapFunction used to jointly transform the two input
@@ -367,63 +379,67 @@ public class DataStream<T> {
/**
* Applies a reduce transformation on preset chunks of the DataStream. The
- * transformation calls a {@link RichGroupReduceFunction} for each tuple
- * batch of the predefined size. Each RichGroupReduceFunction call can
- * return any number of elements including none.
+ * transformation calls a {@link GroupReduceFunction} for each tuple batch
+ * of the predefined size. Each GroupReduceFunction call can return any
+ * number of elements including none. The user can also extend
+ * {@link RichGroupReduceFunction} to gain access to other features provided
+ * by the {@link RichFuntion} interface.
*
*
* @param reducer
- * The RichGroupReduceFunction that is called for each tuple
- * batch.
+ * The GroupReduceFunction that is called for each tuple batch.
* @param batchSize
* The number of tuples grouped together in the batch.
* @param <R>
* output type
* @return The modified DataStream.
*/
- public <R> StreamOperator<R> batchReduce(RichGroupReduceFunction<T, R> reducer, int batchSize) {
+ public <R> StreamOperator<R> batchReduce(GroupReduceFunction<T, R> reducer, int batchSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
- RichGroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<T, R>(reducer,
+ GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<T, R>(reducer,
batchSize));
}
/**
* Applies a reduce transformation on preset "time" chunks of the
- * DataStream. The transformation calls a {@link RichGroupReduceFunction} on
+ * DataStream. The transformation calls a {@link GroupReduceFunction} on
* records received during the predefined time window. The window shifted
- * after each reduce call. Each RichGroupReduceFunction call can return any
- * number of elements including none.
+ * after each reduce call. Each GroupReduceFunction call can return any
+ * number of elements including none.The user can also extend
+ * {@link RichGroupReduceFunction} to gain access to other features provided
+ * by the {@link RichFuntion} interface.
*
*
* @param reducer
- * The RichGroupReduceFunction that is called for each time
- * window.
+ * The GroupReduceFunction that is called for each time window.
* @param windowSize
* The time window to run the reducer on, in milliseconds.
* @param <R>
* output type
* @return The modified DataStream.
*/
- public <R> StreamOperator<R> windowReduce(RichGroupReduceFunction<T, R> reducer, long windowSize) {
+ public <R> StreamOperator<R> windowReduce(GroupReduceFunction<T, R> reducer, long windowSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
- RichGroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<T, R>(reducer,
+ GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<T, R>(reducer,
windowSize));
}
/**
* Applies a Filter transformation on a {@link DataStream}. The
- * transformation calls a {@link RichFilterFunction} for each element of the
+ * transformation calls a {@link FilterFunction} for each element of the
* DataStream and retains only those element for which the function returns
- * true. Elements for which the function returns false are filtered.
+ * true. Elements for which the function returns false are filtered. The
+ * user can also extend {@link RichFilterFunction} to gain access to other
+ * features provided by the {@link RichFuntion} interface.
*
* @param filter
- * The RichFilterFunction that is called for each element of the
+ * The FilterFunction that is called for each element of the
* DataSet.
* @return The filtered DataStream.
*/
- public StreamOperator<T> filter(RichFilterFunction<T> filter) {
+ public StreamOperator<T> filter(FilterFunction<T> filter) {
return addFunction("filter", filter, new FunctionTypeWrapper<T, Tuple, T>(filter,
- RichFilterFunction.class, 0, -1, 0), new FilterInvokable<T>(filter));
+ FilterFunction.class, 0, -1, 0), new FilterInvokable<T>(filter));
}
/**
@@ -745,14 +761,15 @@ public class DataStream<T> {
/**
* Initiates an iterative part of the program that executes multiple times
* and feeds back data streams. The iterative part needs to be closed by
- * calling {@link IterativeDataStream#closeWith(DataStream)}. The data
- * stream given to the {@code closeWith(DataStream)} method is the data
- * stream that will be fed back and used as the input for the iteration
+ * calling {@link IterativeDataStream#closeWith(DataStream)}. The
+ * transformation of this IterativeDataStream will be the iteration head.
+ * The data stream given to the {@code closeWith(DataStream)} method is the
+ * data stream that will be fed back and used as the input for the iteration
* head. Unlike in batch processing by default the output of the iteration
* stream is directed to both to the iteration head and the next component.
* To direct tuples to the iteration head or the output specifically one can
- * use the {@code directTo(OutputSelector)} while referencing the iteration
- * head as 'iterate'.
+ * use the {@code split(OutputSelector)} on the iteration tail while
+ * referencing the iteration head as 'iterate'.
*
* The iteration edge will be partitioned the same way as the first input of
* the iteration head.
@@ -786,8 +803,8 @@ public class DataStream<T> {
* type of the return stream
* @return the data stream constructed
*/
- private <R> StreamOperator<R> addFunction(String functionName,
- final AbstractRichFunction function, TypeSerializerWrapper<T, Tuple, R> typeWrapper,
+ private <R> StreamOperator<R> addFunction(String functionName, final Function function,
+ TypeSerializerWrapper<T, Tuple, R> typeWrapper,
UserTaskInvokable<T, R> functionInvokable) {
DataStream<T> inputStream = this.copy();
@@ -796,7 +813,8 @@ public class DataStream<T> {
try {
jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, typeWrapper,
- functionName, SerializationUtils.serialize(function), degreeOfParallelism);
+ functionName, SerializationUtils.serialize((Serializable) function),
+ degreeOfParallelism);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize user defined function");
}
@@ -816,16 +834,16 @@ public class DataStream<T> {
}
protected <T1, T2, R> StreamOperator<R> addCoFunction(String functionName,
- DataStream<T1> inputStream1, DataStream<T2> inputStream2,
- final AbstractRichFunction function, TypeSerializerWrapper<T1, T2, R> typeWrapper,
- CoInvokable<T1, T2, R> functionInvokable) {
+ DataStream<T1> inputStream1, DataStream<T2> inputStream2, final Function function,
+ TypeSerializerWrapper<T1, T2, R> typeWrapper, CoInvokable<T1, T2, R> functionInvokable) {
StreamOperator<R> returnStream = new TwoInputStreamOperator<T1, T2, R>(environment,
functionName);
try {
jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, typeWrapper,
- functionName, SerializationUtils.serialize(function), degreeOfParallelism);
+ functionName, SerializationUtils.serialize((Serializable) function),
+ degreeOfParallelism);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize user defined function");
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
index 6d63385..798d8fa 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
@@ -51,10 +51,10 @@ public abstract class OutputSelector<T> implements Serializable {
* directTo operator. The tuple will be emitted only to output names which
* are added to the outputs collection.
*
- * @param outputObject
+ * @param value
* Output object for which the output selection should be made.
* @param outputs
* Selected output names should be added to this collection.
*/
- public abstract void select(T outputObject, Collection<String> outputs);
+ public abstract void select(T value, Collection<String> outputs);
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
index 8404a80..d1ef3d0 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
@@ -18,12 +18,13 @@
package org.apache.flink.streaming.api.function.co;
-import org.apache.flink.api.common.functions.AbstractRichFunction;
+import java.io.Serializable;
+import org.apache.flink.api.common.functions.Function;
-public abstract class CoMapFunction<IN1, IN2, OUT> extends AbstractRichFunction {
- private static final long serialVersionUID = 1L;
-
- public abstract OUT map1(IN1 value);
- public abstract OUT map2(IN2 value);
+public interface CoMapFunction<IN1, IN2, OUT> extends Function, Serializable {
+
+ public OUT map1(IN1 value);
+
+ public OUT map2(IN2 value);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoMapFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoMapFunction.java
new file mode 100755
index 0000000..c93fc81
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoMapFunction.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.function.co;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+
+public abstract class RichCoMapFunction<IN1, IN2, OUT> extends AbstractRichFunction implements
+ CoMapFunction<IN1, IN2, OUT> {
+
+ private static final long serialVersionUID = 1L;
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
index 026c18e..4728800 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
@@ -27,7 +27,7 @@ package org.apache.flink.streaming.api.function.sink;
* @param <IN>
* Input tuple type
*/
-public class PrintSinkFunction<IN> extends SinkFunction<IN> {
+public class PrintSinkFunction<IN> implements SinkFunction<IN> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/RichSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/RichSinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/RichSinkFunction.java
new file mode 100755
index 0000000..4bbbdc4
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/RichSinkFunction.java
@@ -0,0 +1,30 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.sink;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+
+public abstract class RichSinkFunction<IN> extends AbstractRichFunction implements SinkFunction<IN> {
+
+ private static final long serialVersionUID = 1L;
+
+ public abstract void invoke(IN value);
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
index 668837f..24f45e8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
@@ -21,11 +21,9 @@ package org.apache.flink.streaming.api.function.sink;
import java.io.Serializable;
-import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.Function;
-public abstract class SinkFunction<IN> extends AbstractRichFunction implements Serializable {
-
- private static final long serialVersionUID = 1L;
+public interface SinkFunction<IN> extends Function, Serializable {
public abstract void invoke(IN value);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
index 774dd63..1cfcfaf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
@@ -32,7 +32,7 @@ import java.util.ArrayList;
* @param <IN>
* Input tuple type
*/
-public abstract class WriteSinkFunction<IN> extends SinkFunction<IN> {
+public abstract class WriteSinkFunction<IN> implements SinkFunction<IN> {
private static final long serialVersionUID = 1L;
protected final String path;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
index 3a732be..6c8cd3a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
@@ -25,7 +25,7 @@ import java.io.IOException;
import org.apache.flink.util.Collector;
-public class FileSourceFunction extends SourceFunction<String> {
+public class FileSourceFunction implements SourceFunction<String> {
private static final long serialVersionUID = 1L;
private final String path;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
index 9cfb2ce..799e700 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
@@ -25,7 +25,7 @@ import java.io.IOException;
import org.apache.flink.util.Collector;
-public class FileStreamFunction extends SourceFunction<String> {
+public class FileStreamFunction implements SourceFunction<String> {
private static final long serialVersionUID = 1L;
private final String path;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
index 89f5182..98e012b 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
@@ -24,7 +24,7 @@ import java.util.Collection;
import org.apache.flink.util.Collector;
-public class FromElementsFunction<T> extends SourceFunction<T> {
+public class FromElementsFunction<T> implements SourceFunction<T> {
private static final long serialVersionUID = 1L;
Iterable<T> iterable;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
index d402374..ad7586c 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
@@ -26,7 +26,7 @@ import org.apache.flink.util.Collector;
* Source Function used to generate the number sequence
*
*/
-public class GenSequenceFunction extends SourceFunction<Long> {
+public class GenSequenceFunction implements SourceFunction<Long> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/RichSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/RichSourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/RichSourceFunction.java
new file mode 100755
index 0000000..94311a1
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/RichSourceFunction.java
@@ -0,0 +1,29 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.source;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+
+public abstract class RichSourceFunction<OUT> extends AbstractRichFunction implements
+ SourceFunction<OUT> {
+
+ private static final long serialVersionUID = 1L;
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
index 44e3387..0bdd7d6 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
@@ -19,12 +19,12 @@
package org.apache.flink.streaming.api.function.source;
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.util.Collector;
+import java.io.Serializable;
-public abstract class SourceFunction<OUT> extends AbstractRichFunction {
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.util.Collector;
- private static final long serialVersionUID = 1L;
+public interface SourceFunction<OUT> extends Function, Serializable {
- public abstract void invoke(Collector<OUT> collector) throws Exception;
+ public void invoke(Collector<OUT> collector) throws Exception;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index b733362..50bdd42 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.api.invokable;
+import org.apache.flink.api.common.functions.RichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
@@ -48,11 +49,15 @@ public class SinkInvokable<IN> extends StreamRecordInvokable<IN, IN> {
@Override
public void open(Configuration parameters) throws Exception {
- sinkFunction.open(parameters);
+ if (sinkFunction instanceof RichFunction) {
+ ((RichFunction) sinkFunction).open(parameters);
+ }
}
@Override
public void close() throws Exception {
- sinkFunction.close();
+ if (sinkFunction instanceof RichFunction) {
+ ((RichFunction) sinkFunction).close();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
index d049bbf..d7710ae 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
+import org.apache.flink.api.common.functions.RichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.function.source.SourceFunction;
@@ -43,11 +44,15 @@ public class SourceInvokable<OUT> extends StreamComponentInvokable<OUT> implemen
@Override
public void open(Configuration parameters) throws Exception {
- sourceFunction.open(parameters);
+ if (sourceFunction instanceof RichFunction) {
+ ((RichFunction) sourceFunction).open(parameters);
+ }
}
@Override
public void close() throws Exception {
- sourceFunction.close();
+ if (sourceFunction instanceof RichFunction) {
+ ((RichFunction) sourceFunction).close();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 61ba5a9..c3c861b 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -22,13 +22,13 @@ package org.apache.flink.streaming.api.invokable.operator;
import java.util.ArrayList;
import java.util.List;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
public class BatchReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private int batchSize;
- public BatchReduceInvokable(RichGroupReduceFunction<IN, OUT> reduceFunction, int batchSize) {
+ public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, int batchSize) {
this.reducer = reduceFunction;
this.batchSize = batchSize;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
index 56ab680..b64f08b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
@@ -19,7 +19,8 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.api.java.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.RichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
@@ -27,9 +28,9 @@ public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
private static final long serialVersionUID = 1L;
- RichFilterFunction<IN> filterFunction;
+ FilterFunction<IN> filterFunction;
- public FilterInvokable(RichFilterFunction<IN> filterFunction) {
+ public FilterInvokable(FilterFunction<IN> filterFunction) {
this.filterFunction = filterFunction;
}
@@ -54,11 +55,15 @@ public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
@Override
public void open(Configuration parameters) throws Exception {
- filterFunction.open(parameters);
+ if (filterFunction instanceof RichFunction) {
+ ((RichFunction) filterFunction).open(parameters);
+ }
}
@Override
public void close() throws Exception {
- filterFunction.close();
+ if (filterFunction instanceof RichFunction) {
+ ((RichFunction) filterFunction).close();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
index 7796230..cc9fcc1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
@@ -19,16 +19,17 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.RichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
- private RichFlatMapFunction<IN, OUT> flatMapper;
+ private FlatMapFunction<IN, OUT> flatMapper;
- public FlatMapInvokable(RichFlatMapFunction<IN, OUT> flatMapper) {
+ public FlatMapInvokable(FlatMapFunction<IN, OUT> flatMapper) {
this.flatMapper = flatMapper;
}
@@ -49,11 +50,15 @@ public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
@Override
public void open(Configuration parameters) throws Exception {
- flatMapper.open(parameters);
+ if (flatMapper instanceof RichFunction) {
+ ((RichFunction) flatMapper).open(parameters);
+ }
}
@Override
public void close() throws Exception {
- flatMapper.close();
+ if (flatMapper instanceof RichFunction) {
+ ((RichFunction) flatMapper).close();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
index 23fc31e..9dbb678 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
@@ -19,16 +19,17 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
- private RichMapFunction<IN, OUT> mapper;
+ private MapFunction<IN, OUT> mapper;
- public MapInvokable(RichMapFunction<IN, OUT> mapper) {
+ public MapInvokable(MapFunction<IN, OUT> mapper) {
this.mapper = mapper;
}
@@ -49,11 +50,15 @@ public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
@Override
public void open(Configuration parameters) throws Exception {
- mapper.open(parameters);
+ if (mapper instanceof RichFunction) {
+ ((RichFunction) mapper).open(parameters);
+ }
}
@Override
public void close() throws Exception {
- mapper.close();
+ if (mapper instanceof RichFunction) {
+ ((RichFunction) mapper).close();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
index 1a402a1..548a298 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
@@ -21,25 +21,30 @@ package org.apache.flink.streaming.api.invokable.operator;
import java.util.Iterator;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.RichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public abstract class StreamReduceInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
- protected RichGroupReduceFunction<IN, OUT> reducer;
+ protected GroupReduceFunction<IN, OUT> reducer;
protected BatchIterator<IN> userIterator;
protected BatchIterable userIterable;
@Override
public void open(Configuration parameters) throws Exception {
userIterable = new BatchIterable();
- reducer.open(parameters);
+ if (reducer instanceof RichFunction) {
+ ((RichFunction) reducer).open(parameters);
+ }
}
@Override
public void close() throws Exception {
- reducer.close();
+ if (reducer instanceof RichFunction) {
+ ((RichFunction) reducer).close();
+ }
}
protected class BatchIterable implements Iterable<IN> {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index 430a68e..309656b 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -22,7 +22,7 @@ package org.apache.flink.streaming.api.invokable.operator;
import java.util.ArrayList;
import java.util.List;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
public class WindowReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
@@ -30,7 +30,7 @@ public class WindowReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OU
volatile boolean isRunning;
boolean window;
- public WindowReduceInvokable(RichGroupReduceFunction<IN, OUT> reduceFunction, long windowSize) {
+ public WindowReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize) {
this.reducer = reduceFunction;
this.windowSize = windowSize;
this.window = true;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
index ac71b22..cd51081 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.api.invokable.operator.co;
+import org.apache.flink.api.common.functions.RichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
@@ -56,12 +57,16 @@ public class CoMapInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
@Override
public void open(Configuration parameters) throws Exception {
- mapper.open(parameters);
+ if (mapper instanceof RichFunction) {
+ ((RichFunction) mapper).open(parameters);
+ }
}
@Override
public void close() throws Exception {
- mapper.close();
+ if (mapper instanceof RichFunction) {
+ ((RichFunction) mapper).close();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
index 0e03915..c06e664 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -25,7 +25,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
@@ -82,7 +82,7 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
}
@SuppressWarnings({ "unchecked", "rawtypes" })
- private void setDeserializers(Object function, Class<? extends AbstractRichFunction> clazz) {
+ private void setDeserializers(Object function, Class<? extends Function> clazz) {
TypeInformation<IN1> inputTypeInfo1 = (TypeInformation<IN1>) typeWrapper
.getInputTypeInfo1();
inputDeserializer1 = new StreamRecordSerializer<IN1>(inputTypeInfo1);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
index 54471ae..2ac6a47 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
@@ -21,22 +21,20 @@ package org.apache.flink.streaming.util.serialization;
import java.io.IOException;
-import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.java.typeutils.TypeExtractor;
-public class FunctionTypeWrapper<IN1, IN2, OUT> extends
- TypeSerializerWrapper<IN1, IN2, OUT> {
+public class FunctionTypeWrapper<IN1, IN2, OUT> extends TypeSerializerWrapper<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
- private AbstractRichFunction function;
- private Class<? extends AbstractRichFunction> functionSuperClass;
+ private Function function;
+ private Class<? extends Function> functionSuperClass;
private int inTypeParameter1;
private int inTypeParameter2;
private int outTypeParameter;
- public FunctionTypeWrapper(AbstractRichFunction function,
- Class<? extends AbstractRichFunction> functionSuperClass, int inTypeParameter1,
- int inTypeParameter2, int outTypeParameter) {
+ public FunctionTypeWrapper(Function function, Class<? extends Function> functionSuperClass,
+ int inTypeParameter1, int inTypeParameter2, int outTypeParameter) {
this.function = function;
this.functionSuperClass = functionSuperClass;
this.inTypeParameter1 = inTypeParameter1;
@@ -54,18 +52,18 @@ public class FunctionTypeWrapper<IN1, IN2, OUT> extends
@Override
protected void setTypeInfo() {
if (inTypeParameter1 != -1) {
- inTypeInfo1 = TypeExtractor.createTypeInfo(functionSuperClass,
- function.getClass(), inTypeParameter1, null, null);
+ inTypeInfo1 = TypeExtractor.createTypeInfo(functionSuperClass, function.getClass(),
+ inTypeParameter1, null, null);
}
if (inTypeParameter2 != -1) {
- inTypeInfo2 = TypeExtractor.createTypeInfo(functionSuperClass,
- function.getClass(), inTypeParameter2, null, null);
+ inTypeInfo2 = TypeExtractor.createTypeInfo(functionSuperClass, function.getClass(),
+ inTypeParameter2, null, null);
}
if (outTypeParameter != -1) {
- outTypeInfo = TypeExtractor.createTypeInfo(functionSuperClass,
- function.getClass(), outTypeParameter, null, null);
+ outTypeInfo = TypeExtractor.createTypeInfo(functionSuperClass, function.getClass(),
+ outTypeParameter, null, null);
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 68403a8..2b3edc2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -64,7 +64,7 @@ public class IterateTest {
}
- public static final class MySink extends SinkFunction<Boolean> {
+ public static final class MySink implements SinkFunction<Boolean> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
index e296733..28cbc6e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
@@ -48,7 +48,7 @@ public class WriteAsCsvTest {
private static List<String> expected4 = new ArrayList<String>();
private static List<String> expected5 = new ArrayList<String>();
- public static final class MySource1 extends SourceFunction<Tuple1<Integer>> {
+ public static final class MySource1 implements SourceFunction<Tuple1<Integer>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
index 0f22262..337ca4e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
@@ -50,7 +50,7 @@ public class WriteAsTextTest {
private static List<String> expected4 = new ArrayList<String>();
private static List<String> expected5 = new ArrayList<String>();
- public static final class MySource1 extends SourceFunction<Tuple1<Integer>> {
+ public static final class MySource1 implements SourceFunction<Tuple1<Integer>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index e2991b4..f2c647f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -51,7 +51,7 @@ public class DirectedOutputTest {
}
}
- private static class EvenSink extends SinkFunction<Long> {
+ private static class EvenSink implements SinkFunction<Long> {
private static final long serialVersionUID = 1L;
@@ -61,7 +61,7 @@ public class DirectedOutputTest {
}
}
- private static class OddSink extends SinkFunction<Long> {
+ private static class OddSink implements SinkFunction<Long> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
index c23c9a7..49f4509 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
@@ -42,8 +42,8 @@ public class BatchReduceTest {
private static final int PARALlELISM = 1;
private static final long MEMORYSIZE = 32;
- public static final class MyBatchReduce extends
- RichGroupReduceFunction<Tuple1<Double>, Tuple1<Double>> {
+ public static final class MyBatchReduce implements
+ GroupReduceFunction<Tuple1<Double>, Tuple1<Double>> {
private static final long serialVersionUID = 1L;
@Override
@@ -62,7 +62,7 @@ public class BatchReduceTest {
}
}
- public static final class MySink extends SinkFunction<Tuple1<Double>> {
+ public static final class MySink implements SinkFunction<Tuple1<Double>> {
private static final long serialVersionUID = 1L;
@Override
@@ -72,7 +72,7 @@ public class BatchReduceTest {
}
- public static final class MySource extends SourceFunction<Tuple1<Double>> {
+ public static final class MySource implements SourceFunction<Tuple1<Double>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
index 2c3f480..82a5f89 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
@@ -39,7 +39,7 @@ public class CoMapTest implements Serializable {
private static Set<String> result;
private static Set<String> expected = new HashSet<String>();
- private final static class EmptySink extends SinkFunction<Boolean> {
+ private final static class EmptySink implements SinkFunction<Boolean> {
private static final long serialVersionUID = 1L;
@Override
@@ -47,7 +47,7 @@ public class CoMapTest implements Serializable {
}
}
- private final static class MyCoMap extends
+ private final static class MyCoMap implements
CoMapFunction<String, Integer, Boolean> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
index 2d4fe7a..0cba0bf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
@@ -23,7 +23,7 @@ import java.io.Serializable;
import java.util.HashSet;
import java.util.Set;
-import org.apache.flink.api.java.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.FilterFunction;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
@@ -36,7 +36,7 @@ public class FilterTest implements Serializable {
private static Set<Integer> set = new HashSet<Integer>();
- private static class MySink extends SinkFunction<Integer> {
+ private static class MySink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -45,7 +45,7 @@ public class FilterTest implements Serializable {
}
}
- static class MyFilter extends RichFilterFunction<Integer> {
+ static class MyFilter implements FilterFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
[04/51] [abbrv] git commit: [streaming] API javadoc +
StreamRecordSerializer update
Posted by se...@apache.org.
[streaming] API javadoc + StreamRecordSerializer update
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/5dbf8152
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/5dbf8152
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/5dbf8152
Branch: refs/heads/master
Commit: 5dbf81521d6c8b5b7937e041a9f9e8b0327dc345
Parents: 11f62c1
Author: gyfora <gy...@gmail.com>
Authored: Mon Jul 21 12:49:56 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:14:13 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 18 ++++++++++++++----
.../api/streamcomponent/StreamIterationSink.java | 2 +-
.../streaming/api/streamrecord/StreamRecord.java | 4 ++--
.../api/streamrecord/StreamRecordSerializer.java | 3 +--
.../org/apache/flink/streaming/api/PrintTest.java | 2 +-
5 files changed, 19 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5dbf8152/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index d327fad..e67ac0e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -129,9 +129,19 @@ public class DataStream<T extends Tuple> {
public String getId() {
return id;
}
-
- public DataStream<T> setMutability(boolean isMutable){
- environment.setMutability(this,isMutable);
+
+ /**
+ * Sets the mutability of the operator represented by the DataStream. If the
+ * operator is set to mutable, the tuples received in the user defined
+ * functions, will be reused after the function call. Setting an operator to
+ * mutable greatly reduces garbage collection overhead and thus scalability.
+ *
+ * @param isMutable
+ * The mutability of the operator.
+ * @return The DataStream with mutability set.
+ */
+ public DataStream<T> setMutability(boolean isMutable) {
+ environment.setMutability(this, isMutable);
return this;
}
@@ -400,7 +410,7 @@ public class DataStream<T extends Tuple> {
public DataStream<T> print() {
return environment.print(new DataStream<T>(this));
}
-
+
/**
* Writes a DataStream to the file specified by path in text format. For
* every element of the DataStream the result of {@link Object#toString()}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5dbf8152/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
index b9d44c3..b92e031 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
@@ -79,7 +79,7 @@ public class StreamIterationSink<IN extends Tuple> extends AbstractStreamCompone
while ((reuse = inputIter.next(reuse)) != null) {
pushToQueue(reuse);
// TODO: Fix object reuse for iteration
- reuse = inTupleSerializer.createInstance().setId(0);
+ reuse = inTupleSerializer.createInstance();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5dbf8152/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
index 5021013..9de2f38 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
@@ -31,13 +31,13 @@ import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
public class StreamRecord<T extends Tuple> implements Serializable {
private static final long serialVersionUID = 1L;
- protected UID uid;
+ private UID uid;
private T tuple;
protected TupleSerializer<T> tupleSerializer;
public StreamRecord() {
-
+ uid = new UID();
}
/**
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5dbf8152/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
index b367333..66c6751 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
@@ -50,9 +50,8 @@ public final class StreamRecordSerializer<T extends Tuple> extends TypeSerialize
public StreamRecord<T> createInstance() {
try {
@SuppressWarnings("unchecked")
- StreamRecord<T> t = StreamRecord.class.newInstance().setId(0);
+ StreamRecord<T> t = StreamRecord.class.newInstance();
t.setTuple(tupleSerializer.createInstance());
-
return t;
} catch (Exception e) {
throw new RuntimeException("Cannot instantiate StreamRecord.", e);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5dbf8152/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
index c70cca3..1c43a66 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
@@ -78,7 +78,7 @@ public class PrintTest {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
env.generateSequence(1, 10).print();
- env.execute();
+ env.executeTest(MEMORYSIZE);
}
[38/51] [abbrv] [streaming] API update with more differentiated
DataStream types and javadoc + several fixes
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
new file mode 100755
index 0000000..8bcde44
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
@@ -0,0 +1,70 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+/**
+ * The SplitDataStream represents an operator that has been split using an
+ * {@link OutputSelector}. Named outputs can be selected using the
+ * {@link #select} function.
+ *
+ * @param <T>
+ * The type of the output.
+ */
+public class SplitDataStream<T> {
+
+ DataStream<T> dataStream;
+
+ protected SplitDataStream(DataStream<T> dataStream) {
+ this.dataStream = dataStream.copy();
+ }
+
+ /**
+ * Sets the output names for which the next operator will receive values.
+ *
+ * @param outputNames
+ * The output names for which the operator will receive the
+ * input.
+ * @return Returns the modified DataStream
+ */
+ public DataStream<T> select(String... outputNames) {
+ DataStream<T> returnStream = selectOutput(outputNames[0]);
+ for (int i = 1; i < outputNames.length; i++) {
+ if (outputNames[i] == "") {
+ throw new IllegalArgumentException("User defined name must not be empty string");
+ }
+
+ returnStream = connectWithNames(returnStream, selectOutput(outputNames[i]));
+ }
+ return returnStream;
+ }
+
+ private DataStream<T> connectWithNames(DataStream<T> stream1, DataStream<T> stream2) {
+ ConnectedDataStream<T> returnStream = new ConnectedDataStream<T>(stream1.copy());
+ returnStream.connectedStreams.add(stream2.copy());
+ return returnStream;
+ }
+
+ private DataStream<T> selectOutput(String outputName) {
+ DataStream<T> returnStream = dataStream.copy();
+ returnStream.userDefinedName = outputName;
+ return returnStream;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
new file mode 100755
index 0000000..16e84bb
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
@@ -0,0 +1,40 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.environment;
+
+import org.apache.flink.streaming.util.ClusterUtil;
+
+public class LocalStreamEnvironment extends StreamExecutionEnvironment {
+
+ /**
+ * Executes the JobGraph of the on a mini cluster of CLusterUtil.
+ *
+ */
+ @Override
+ public void execute() {
+ ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(), getExecutionParallelism());
+ }
+
+ public void executeTest(long memorySize) {
+ ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(), getExecutionParallelism(),
+ memorySize);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
new file mode 100644
index 0000000..19a2d48
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -0,0 +1,107 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.environment;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.flink.client.program.Client;
+import org.apache.flink.client.program.JobWithJars;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+
+public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
+ private static final Log log = LogFactory.getLog(RemoteStreamEnvironment.class);
+
+ private String host;
+ private int port;
+ private String[] jarFiles;
+
+ /**
+ * Creates a new RemoteStreamEnvironment that points to the master
+ * (JobManager) described by the given host name and port.
+ *
+ * @param host
+ * The host name or address of the master (JobManager), where the
+ * program should be executed.
+ * @param port
+ * The port of the master (JobManager), where the program should
+ * be executed.
+ * @param jarFiles
+ * The JAR files with code that needs to be shipped to the
+ * cluster. If the program uses user-defined functions,
+ * user-defined input formats, or any libraries, those must be
+ * provided in the JAR files.
+ */
+ public RemoteStreamEnvironment(String host, int port, String... jarFiles) {
+ if (host == null) {
+ throw new NullPointerException("Host must not be null.");
+ }
+
+ if (port < 1 || port >= 0xffff) {
+ throw new IllegalArgumentException("Port out of range");
+ }
+
+ this.host = host;
+ this.port = port;
+ this.jarFiles = jarFiles;
+ }
+
+ @Override
+ public void execute() {
+ if (log.isInfoEnabled()) {
+ log.info("Running remotely at " + host + ":" + port);
+ }
+
+ JobGraph jobGraph = jobGraphBuilder.getJobGraph();
+
+ for (int i = 0; i < jarFiles.length; i++) {
+ File file = new File(jarFiles[i]);
+ try {
+ JobWithJars.checkJarFile(file);
+ } catch (IOException e) {
+ throw new RuntimeException("Problem with jar file " + file.getAbsolutePath(), e);
+ }
+ jobGraph.addJar(new Path(file.getAbsolutePath()));
+ }
+
+ Configuration configuration = jobGraph.getJobConfiguration();
+ Client client = new Client(new InetSocketAddress(host, port), configuration);
+
+ try {
+ client.run(jobGraph, true);
+ } catch (ProgramInvocationException e) {
+ throw new RuntimeException("Cannot execute job due to ProgramInvocationException", e);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "Remote Environment (" + this.host + ":" + this.port + " - DOP = "
+ + (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) + ")";
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
new file mode 100644
index 0000000..3773d8e
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -0,0 +1,413 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.environment;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+import org.apache.commons.lang3.SerializationException;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.JobGraphBuilder;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.function.source.FileSourceFunction;
+import org.apache.flink.streaming.api.function.source.FileStreamFunction;
+import org.apache.flink.streaming.api.function.source.FromElementsFunction;
+import org.apache.flink.streaming.api.function.source.GenSequenceFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.streaming.api.invokable.SourceInvokable;
+import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
+import org.apache.flink.streaming.util.serialization.ObjectTypeWrapper;
+
+/**
+ * {@link ExecutionEnvironment} for streaming jobs. An instance of it is
+ * necessary to construct streaming topologies.
+ *
+ */
+public abstract class StreamExecutionEnvironment {
+
+ /**
+ * The environment of the context (local by default, cluster if invoked
+ * through command line)
+ */
+ private static StreamExecutionEnvironment contextEnvironment;
+
+ /** flag to disable local executor when using the ContextEnvironment */
+ private static boolean allowLocalExecution = true;
+
+ private static int defaultLocalDop = Runtime.getRuntime().availableProcessors();
+
+ private int degreeOfParallelism = 1;
+
+ private int executionParallelism = -1;
+
+ protected JobGraphBuilder jobGraphBuilder;
+
+ // --------------------------------------------------------------------------------------------
+ // Constructor and Properties
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Constructor for creating StreamExecutionEnvironment
+ */
+ protected StreamExecutionEnvironment() {
+ jobGraphBuilder = new JobGraphBuilder("jobGraph");
+ }
+
+ public int getExecutionParallelism() {
+ return executionParallelism == -1 ? degreeOfParallelism : executionParallelism;
+ }
+
+ /**
+ * Gets the degree of parallelism with which operation are executed by
+ * default. Operations can individually override this value to use a
+ * specific degree of parallelism via {@link DataStream#setParallelism}.
+ *
+ * @return The degree of parallelism used by operations, unless they
+ * override that value.
+ */
+ public int getDegreeOfParallelism() {
+ return this.degreeOfParallelism;
+ }
+
+ /**
+ * Sets the degree of parallelism (DOP) for operations executed through this
+ * environment. Setting a DOP of x here will cause all operators (such as
+ * map, batchReduce) to run with x parallel instances. This method overrides
+ * the default parallelism for this environment. The
+ * {@link LocalStreamEnvironment} uses by default a value equal to the
+ * number of hardware contexts (CPU cores / threads). When executing the
+ * program via the command line client from a JAR file, the default degree
+ * of parallelism is the one configured for that setup.
+ *
+ * @param degreeOfParallelism
+ * The degree of parallelism
+ */
+ protected void setDegreeOfParallelism(int degreeOfParallelism) {
+ if (degreeOfParallelism < 1) {
+ throw new IllegalArgumentException("Degree of parallelism must be at least one.");
+ }
+ this.degreeOfParallelism = degreeOfParallelism;
+ }
+
+ /**
+ * Sets the number of hardware contexts (CPU cores / threads) used when
+ * executed in {@link LocalStreamEnvironment}.
+ *
+ * @param degreeOfParallelism
+ * The degree of parallelism in local environment
+ */
+ public void setExecutionParallelism(int degreeOfParallelism) {
+ if (degreeOfParallelism < 1) {
+ throw new IllegalArgumentException("Degree of parallelism must be at least one.");
+ }
+
+ this.executionParallelism = degreeOfParallelism;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Data stream creations
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Creates a DataStream that represents the Strings produced by reading the
+ * given file line wise. The file will be read with the system's default
+ * character set.
+ *
+ * @param filePath
+ * The path of the file, as a URI (e.g.,
+ * "file:///some/local/file" or "hdfs://host:port/file/path").
+ * @return The DataStream representing the text file.
+ */
+ public DataStreamSource<String> readTextFile(String filePath) {
+ return addSource(new FileSourceFunction(filePath), 1);
+ }
+
+ public DataStreamSource<String> readTextFile(String filePath, int parallelism) {
+ return addSource(new FileSourceFunction(filePath), parallelism);
+ }
+
+ /**
+ * Creates a DataStream that represents the Strings produced by reading the
+ * given file line wise multiple times(infinite). The file will be read with
+ * the system's default character set.
+ *
+ * @param filePath
+ * The path of the file, as a URI (e.g.,
+ * "file:///some/local/file" or "hdfs://host:port/file/path").
+ * @return The DataStream representing the text file.
+ */
+ public DataStreamSource<String> readTextStream(String filePath) {
+ return addSource(new FileStreamFunction(filePath), 1);
+ }
+
+ public DataStreamSource<String> readTextStream(String filePath, int parallelism) {
+ return addSource(new FileStreamFunction(filePath), parallelism);
+ }
+
+ /**
+ * Creates a new DataStream that contains the given elements. The elements
+ * must all be of the same type, for example, all of the String or Integer.
+ * The sequence of elements must not be empty. Furthermore, the elements
+ * must be serializable (as defined in java.io.Serializable), because the
+ * execution environment may ship the elements into the cluster.
+ *
+ * @param data
+ * The collection of elements to create the DataStream from.
+ * @param <OUT>
+ * type of the returned stream
+ * @return The DataStream representing the elements.
+ */
+ public <OUT extends Serializable> DataStreamSource<OUT> fromElements(OUT... data) {
+ DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(
+ this, "elements");
+
+ try {
+ SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
+ jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
+ new ObjectTypeWrapper<OUT, Tuple, OUT>(data[0], null, data[0]), "source",
+ SerializationUtils.serialize(function), 1);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize elements");
+ }
+ return returnStream;
+ }
+
+ /**
+ * Creates a DataStream from the given non-empty collection. The type of the
+ * DataStream is that of the elements in the collection. The elements need
+ * to be serializable (as defined by java.io.Serializable), because the
+ * framework may move the elements into the cluster if needed.
+ *
+ * @param data
+ * The collection of elements to create the DataStream from.
+ * @param <OUT>
+ * type of the returned stream
+ * @return The DataStream representing the elements.
+ */
+ @SuppressWarnings("unchecked")
+ public <OUT extends Serializable> DataStreamSource<OUT> fromCollection(Collection<OUT> data) {
+ DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements");
+
+ if (data.isEmpty()) {
+ throw new RuntimeException("Collection must not be empty");
+ }
+
+ try {
+ SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
+
+ jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(
+ new FromElementsFunction<OUT>(data)), new ObjectTypeWrapper<OUT, Tuple, OUT>(
+ (OUT) data.toArray()[0], null, (OUT) data.toArray()[0]), "source",
+ SerializationUtils.serialize(function), 1);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize collection");
+ }
+
+ return returnStream;
+ }
+
+ /**
+ * Creates a new DataStream that contains a sequence of numbers.
+ *
+ * @param from
+ * The number to start at (inclusive).
+ * @param to
+ * The number to stop at (inclusive)
+ * @return A DataStrean, containing all number in the [from, to] interval.
+ */
+ public DataStreamSource<Long> generateSequence(long from, long to) {
+ return addSource(new GenSequenceFunction(from, to), 1);
+ }
+
+ /**
+ * Ads a data source thus opening a {@link DataStream}.
+ *
+ * @param function
+ * the user defined function
+ * @param parallelism
+ * number of parallel instances of the function
+ * @param <OUT>
+ * type of the returned stream
+ * @return the data stream constructed
+ */
+ public <OUT> DataStreamSource<OUT> addSource(SourceFunction<OUT> function, int parallelism) {
+ DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "source");
+
+ try {
+ jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
+ new FunctionTypeWrapper<OUT, Tuple, OUT>(function, SourceFunction.class, 0, -1,
+ 0), "source", SerializationUtils.serialize(function), parallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize SourceFunction");
+ }
+
+ return returnStream;
+ }
+
+ public <OUT> DataStreamSource<OUT> addSource(SourceFunction<OUT> sourceFunction) {
+ return addSource(sourceFunction, 1);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Instantiation of Execution Contexts
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Creates an execution environment that represents the context in which the
+ * program is currently executed. If the program is invoked standalone, this
+ * method returns a local execution environment, as returned by
+ * {@link #createLocalEnvironment()}.
+ *
+ * @return The execution environment of the context in which the program is
+ * executed.
+ */
+ public static StreamExecutionEnvironment getExecutionEnvironment() {
+ return contextEnvironment == null ? createLocalEnvironment() : contextEnvironment;
+ }
+
+ /**
+ * Creates a {@link LocalStreamEnvironment}. The local execution environment
+ * will run the program in a multi-threaded fashion in the same JVM as the
+ * environment was created in. The default degree of parallelism of the
+ * local environment is the number of hardware contexts (CPU cores /
+ * threads), unless it was specified differently by
+ * {@link #setDegreeOfParallelism(int)}.
+ *
+ * @return A local execution environment.
+ */
+ public static LocalStreamEnvironment createLocalEnvironment() {
+ return createLocalEnvironment(defaultLocalDop);
+ }
+
+ /**
+ * Creates a {@link LocalStreamEnvironment}. The local execution environment
+ * will run the program in a multi-threaded fashion in the same JVM as the
+ * environment was created in. It will use the degree of parallelism
+ * specified in the parameter.
+ *
+ * @param degreeOfParallelism
+ * The degree of parallelism for the local environment.
+ * @return A local execution environment with the specified degree of
+ * parallelism.
+ */
+ public static LocalStreamEnvironment createLocalEnvironment(int degreeOfParallelism) {
+ LocalStreamEnvironment lee = new LocalStreamEnvironment();
+ lee.setDegreeOfParallelism(degreeOfParallelism);
+ return lee;
+ }
+
+ // TODO:fix cluster default parallelism
+ /**
+ * Creates a {@link RemoteStreamEnvironment}. The remote environment sends
+ * (parts of) the program to a cluster for execution. Note that all file
+ * paths used in the program must be accessible from the cluster. The
+ * execution will use no parallelism, unless the parallelism is set
+ * explicitly via {@link #setDegreeOfParallelism}.
+ *
+ * @param host
+ * The host name or address of the master (JobManager), where the
+ * program should be executed.
+ * @param port
+ * The port of the master (JobManager), where the program should
+ * be executed.
+ * @param jarFiles
+ * The JAR files with code that needs to be shipped to the
+ * cluster. If the program uses user-defined functions,
+ * user-defined input formats, or any libraries, those must be
+ * provided in the JAR files.
+ * @return A remote environment that executes the program on a cluster.
+ */
+ public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port,
+ String... jarFiles) {
+ return new RemoteStreamEnvironment(host, port, jarFiles);
+ }
+
+ /**
+ * Creates a {@link RemoteStreamEnvironment}. The remote environment sends
+ * (parts of) the program to a cluster for execution. Note that all file
+ * paths used in the program must be accessible from the cluster. The
+ * execution will use the specified degree of parallelism.
+ *
+ * @param host
+ * The host name or address of the master (JobManager), where the
+ * program should be executed.
+ * @param port
+ * The port of the master (JobManager), where the program should
+ * be executed.
+ * @param degreeOfParallelism
+ * The degree of parallelism to use during the execution.
+ * @param jarFiles
+ * The JAR files with code that needs to be shipped to the
+ * cluster. If the program uses user-defined functions,
+ * user-defined input formats, or any libraries, those must be
+ * provided in the JAR files.
+ * @return A remote environment that executes the program on a cluster.
+ */
+ public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port,
+ int degreeOfParallelism, String... jarFiles) {
+ RemoteStreamEnvironment rec = new RemoteStreamEnvironment(host, port, jarFiles);
+ rec.setDegreeOfParallelism(degreeOfParallelism);
+ return rec;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Methods to control the context and local environments for execution from
+ // packaged programs
+ // --------------------------------------------------------------------------------------------
+
+ protected static void initializeContextEnvironment(StreamExecutionEnvironment ctx) {
+ contextEnvironment = ctx;
+ }
+
+ protected static boolean isContextEnvironmentSet() {
+ return contextEnvironment != null;
+ }
+
+ protected static void disableLocalExecution() {
+ allowLocalExecution = false;
+ }
+
+ public static boolean localExecutionIsAllowed() {
+ return allowLocalExecution;
+ }
+
+ /**
+ * Triggers the program execution. The environment will execute all parts of
+ * the program that have resulted in a "sink" operation. Sink operations are
+ * for example printing results or forwarding them to a message queue.
+ * <p>
+ * The program execution will be logged and displayed with a generated
+ * default name.
+ **/
+ public abstract void execute();
+
+ /**
+ * Getter of the {@link JobGraphBuilder} of the streaming job.
+ *
+ * @return jobgraph
+ */
+ public JobGraphBuilder getJobGraphBuilder() {
+ return jobGraphBuilder;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
index d1ef3d0..cc14e2c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
@@ -22,6 +22,17 @@ import java.io.Serializable;
import org.apache.flink.api.common.functions.Function;
+/**
+ * A CoMapFunction represents a Map transformation with two different input
+ * types.
+ *
+ * @param <IN1>
+ * Type of the first input.
+ * @param <IN2>
+ * Type of the second input.
+ * @param <OUT>
+ * Output type.
+ */
public interface CoMapFunction<IN1, IN2, OUT> extends Function, Serializable {
public OUT map1(IN1 value);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoMapFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoMapFunction.java
index c93fc81..1468181 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoMapFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoMapFunction.java
@@ -19,7 +19,20 @@
package org.apache.flink.streaming.api.function.co;
import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+/**
+ * A RichCoMapFunction represents a Map transformation with two different input
+ * types. In addition to that the user can use the features provided by the
+ * {@link RichFunction} interface.
+ *
+ * @param <IN1>
+ * Type of the first input.
+ * @param <IN2>
+ * Type of the second input.
+ * @param <OUT>
+ * Output type.
+ */
public abstract class RichCoMapFunction<IN1, IN2, OUT> extends AbstractRichFunction implements
CoMapFunction<IN1, IN2, OUT> {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index e50803f..76dee5d 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -141,7 +141,7 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
}
outputs.add(output);
- String outputName = configuration.getOutputName(outputNumber);
+ List<String> outputName = configuration.getOutputName(outputNumber);
if (collector != null) {
collector.addOutput(output, outputName);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 2b3edc2..5872da9 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -25,6 +25,10 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.IterativeDataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
index 67dce9d..83c98fc 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
@@ -19,6 +19,8 @@
package org.apache.flink.streaming.api;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.log4j.Level;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
index 28cbc6e..8e9475a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
@@ -29,6 +29,9 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
index 337ca4e..122979d 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
@@ -29,6 +29,9 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index f2c647f..08387f9 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -27,9 +27,10 @@ import java.util.Collection;
import java.util.HashSet;
import org.apache.flink.api.java.functions.RichMapFunction;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.SplitDataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.log4j.Level;
@@ -94,13 +95,11 @@ public class DirectedOutputTest {
public void directOutputTest() throws Exception {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
- StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+ LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
SplitDataStream<Long> s = env.generateSequence(1, 6).split(new MySelector());
DataStream<Long> ds1 = s.select("ds1").shuffle().map(new PlusTwo()).addSink(new EvenSink());
DataStream<Long> ds2 = s.select("ds2").map(new PlusTwo()).addSink(new OddSink());
- DataStream<Long> ds3 = s.map(new PlusTwo()).addSink(new OddSink());
-
- env.execute();
+ env.executeTest(32);
HashSet<Long> expectedEven = new HashSet<Long>(Arrays.asList(4L, 6L, 8L));
HashSet<Long> expectedOdd = new HashSet<Long>(Arrays.asList(3L, 5L, 7L));
@@ -114,7 +113,7 @@ public class DirectedOutputTest {
public void directNamingTest() {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
- StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+ LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
SplitDataStream<Long> s = env.generateSequence(1, 10).split(new MySelector());
try {
s.select("ds2").connectWith(s.select("ds1"));
@@ -122,20 +121,6 @@ public class DirectedOutputTest {
} catch (Exception e) {
// Exception thrown
}
- try {
- s.shuffle().connectWith(s.select("ds1"));
- fail();
- } catch (Exception e) {
- // Exception thrown
- }
- try {
- s.select("ds2").connectWith(s);
- fail();
- } catch (Exception e) {
- // Exception thrown
- }
- s.connectWith(s);
- s.select("ds2").connectWith(s.select("ds2"));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
index 7c7f593..72b09c9 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
@@ -21,6 +21,8 @@ package org.apache.flink.streaming.api.collector;
import static org.junit.Assert.assertArrayEquals;
+import java.util.ArrayList;
+
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamcomponent.MockRecordWriter;
@@ -34,14 +36,15 @@ public class StreamCollectorTest {
public void testCollect() {
MockRecordWriter recWriter = MockRecordWriterFactory.create();
- StreamCollector<Tuple1<Integer>> collector = new StreamCollector<Tuple1<Integer>>(2, new SerializationDelegate<StreamRecord<Tuple1<Integer>>>(null));
- collector.addOutput(recWriter, null);
+ StreamCollector<Tuple1<Integer>> collector = new StreamCollector<Tuple1<Integer>>(2,
+ new SerializationDelegate<StreamRecord<Tuple1<Integer>>>(null));
+ collector.addOutput(recWriter, new ArrayList<String>());
collector.collect(new Tuple1<Integer>(3));
collector.collect(new Tuple1<Integer>(4));
collector.collect(new Tuple1<Integer>(5));
collector.collect(new Tuple1<Integer>(6));
- assertArrayEquals(new Integer[] {3, 4, 5, 6}, recWriter.emittedRecords.toArray());
+ assertArrayEquals(new Integer[] { 3, 4, 5, 6 }, recWriter.emittedRecords.toArray());
}
@Test
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
index 49f4509..c91878b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -25,9 +25,9 @@ import java.util.ArrayList;
import org.apache.flink.api.common.functions.GroupReduceFunction;
import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.LogUtils;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
index 82a5f89..020dae4 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
@@ -23,9 +23,9 @@ import java.io.Serializable;
import java.util.HashSet;
import java.util.Set;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
@@ -47,8 +47,7 @@ public class CoMapTest implements Serializable {
}
}
- private final static class MyCoMap implements
- CoMapFunction<String, Integer, Boolean> {
+ private final static class MyCoMap implements CoMapFunction<String, Integer, Boolean> {
private static final long serialVersionUID = 1L;
@Override
@@ -86,10 +85,8 @@ public class CoMapTest implements Serializable {
DataStream<String> ds3 = env.fromElements("a", "b");
@SuppressWarnings({ "unused", "unchecked" })
- DataStream<Boolean> ds4 = env.fromElements("c").connectWith(ds3)
- .coMapWith(new MyCoMap(),
-
- ds2).addSink(new EmptySink());
+ DataStream<Boolean> ds4 = env.fromElements("c").connectWith(ds3).co(ds2).map(new MyCoMap())
+ .addSink(new EmptySink());
env.executeTest(32);
Assert.assertEquals(expected, result);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
index 0cba0bf..ec625e9 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
@@ -24,7 +24,7 @@ import java.util.HashSet;
import java.util.Set;
import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.log4j.Level;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
index ca9c1cb..06f8447 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
@@ -26,9 +26,9 @@ import java.util.HashSet;
import java.util.Set;
import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
index 73185df..0c59864 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
@@ -28,9 +28,9 @@ import java.util.List;
import java.util.Set;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.LogUtils;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
index 7d78d8f..545169d 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
@@ -27,9 +27,9 @@ import java.util.Map;
import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
index c5b51e6..b3cb9dc 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
@@ -21,8 +21,8 @@ package org.apache.flink.streaming.examples.basictopology;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
index bb5cc07..4f98f5a 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
@@ -24,8 +24,8 @@ import java.util.Random;
import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansLocal.java
index f9be802..e132867 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansLocal.java
@@ -19,7 +19,7 @@
package org.apache.flink.streaming.examples.iterative.kmeans;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
public class KMeansLocal {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankLocal.java
index 3d4a451..c84b86e 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankLocal.java
@@ -19,7 +19,7 @@
package org.apache.flink.streaming.examples.iterative.pagerank;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
public class PageRankLocal {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPLocal.java
index 528511d..87f4a75 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPLocal.java
@@ -19,7 +19,7 @@
package org.apache.flink.streaming.examples.iterative.sssp;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
public class SSSPLocal {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
index cd29e7b..08738a2 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
@@ -19,11 +19,10 @@
package org.apache.flink.streaming.examples.join;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.log4j.Level;
-
import org.apache.flink.api.java.tuple.Tuple3;
public class JoinLocal {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index 6264cb9..4e9022b 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -21,8 +21,8 @@ package org.apache.flink.streaming.examples.ml;
import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
index 366e7b5..078b514 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
@@ -27,8 +27,8 @@ import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
index 2b182c8..83cdd52 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
@@ -21,8 +21,8 @@ package org.apache.flink.streaming.examples.window.join;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.log4j.Level;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d56d48f1/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
index f77ab37..fc31930 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
@@ -19,10 +19,9 @@
package org.apache.flink.streaming.examples.wordcount;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.TestDataUtil;
-
import org.apache.flink.api.java.tuple.Tuple2;
public class WordCountLocal {
[50/51] [abbrv] git commit: [streaming] Fix LICENSE file for
streaming project. Minor merge fixes.
Posted by se...@apache.org.
[streaming] Fix LICENSE file for streaming project. Minor merge fixes.
This closes #72
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/0163cfa1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/0163cfa1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/0163cfa1
Branch: refs/heads/master
Commit: 0163cfa11d64b641718a70aa30b7b896cdc077ba
Parents: 35cf874
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Aug 18 17:00:39 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 17:34:09 2014 +0200
----------------------------------------------------------------------
LICENSE | 19 +++++++++++
NOTICE | 8 +++++
.../flink-streaming-connectors/pom.xml | 5 ++-
.../flink-streaming-core/pom.xml | 7 -----
.../environment/RemoteStreamEnvironment.java | 2 +-
.../api/streamcomponent/StreamRecordWriter.java | 19 +++++++++++
.../flink/streaming/util/ClusterUtil.java | 6 ++--
.../flink-streaming-examples/pom.xml | 6 ----
flink-addons/flink-streaming/pom.xml | 33 +-------------------
flink-core/pom.xml | 8 -----
flink-dist/pom.xml | 10 +++---
flink-dist/src/main/flink-bin/LICENSE | 21 ++++++++++++-
flink-dist/src/main/flink-bin/NOTICE | 8 +++++
pom.xml | 8 ++++-
14 files changed, 93 insertions(+), 67 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index c46c750..15caf5a 100644
--- a/LICENSE
+++ b/LICENSE
@@ -227,6 +227,8 @@ under the Apache License (v 2.0):
- Apache Avro (http://avro.apache.org)
- Apache Hadoop (http://hadoop.apache.org)
- Apache Derby (http://db.apache.org/derby/)
+ - Apache Kafka (http://kafka.apache.org)
+ - Apache Flume (http://flume.apache.org)
- Google Guava (https://code.google.com/p/guava-libraries/)
- Netty (http://netty.io)
- Powermock (http://www.powermock.org)
@@ -236,6 +238,8 @@ under the Apache License (v 2.0):
- ScalaTest (http://www.scalatest.org)
- StartBootstrap (http://startbootstrap.com)
- CHAP Links Library Timeline (http://almende.github.io/chap-links-library/)
+ - Twitter Hosebird Client (hbc) (https://github.com/twitter/hbc)
+
-----------------------------------------------------------------------
Eclipse Public License - v 1.0
@@ -337,6 +341,21 @@ POSSIBILITY OF SUCH DAMAGE.
-----------------------------------------------------------------------
+ Mozilla Public License - v 1.1
+-----------------------------------------------------------------------
+
+The Apache Flink project depends on and/or bundles the following components
+under the Mozilla Public License (v 1.1)
+
+ - RabbitMQ (http://www.rabbitmq.com)
+ The Initial Developer of the Original Code is GoPivotal,Ltd.
+ Copyright (c) 2007-2013 GoPivotal, Inc. All Rights Reserved.
+
+You may obtain a copy of the Mozilla Public License (v 1.1) at
+http://www.mozilla.org/MPL/
+
+
+-----------------------------------------------------------------------
The Open Font License
-----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 00ebeda..cf03416 100644
--- a/NOTICE
+++ b/NOTICE
@@ -89,6 +89,14 @@ projects: See https://issues.apache.org/jira/browse/HBASE-3374)
-----------------------------------------------------------------------
+ Apache Flume
+-----------------------------------------------------------------------
+
+Portions of this software were developed at
+Cloudera, Inc. (http://www.cloudera.com/).
+
+
+-----------------------------------------------------------------------
Amazon Web Services SDK for Java
-----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
index 23e3fef..4424c91 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
@@ -66,20 +66,19 @@ under the License.
<version>1.5.0</version>
</dependency>
- <dependency>
+ <dependency>
<groupId>com.twitter</groupId>
<artifactId>hbc-core</artifactId>
<version>2.2.0</version>
</dependency>
- </dependencies>
+ </dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
- <version>2.4</version>
<executions>
<execution>
<goals>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-addons/flink-streaming/flink-streaming-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/pom.xml b/flink-addons/flink-streaming/flink-streaming-core/pom.xml
index 73b8158..3877930 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-core/pom.xml
@@ -37,12 +37,6 @@ under the License.
<dependencies>
<dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>commons-lang3</artifactId>
- <version>3.1</version>
- </dependency>
-
- <dependency>
<groupId>org.apache.sling</groupId>
<artifactId>org.apache.sling.commons.json</artifactId>
<version>2.0.6</version>
@@ -55,7 +49,6 @@ under the License.
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
- <version>2.4</version>
<executions>
<execution>
<goals>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index ec6bc6f..f6495e2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -89,7 +89,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
}
Configuration configuration = jobGraph.getJobConfiguration();
- Client client = new Client(new InetSocketAddress(host, port), configuration);
+ Client client = new Client(new InetSocketAddress(host, port), configuration, getClass().getClassLoader());
try {
client.run(jobGraph, true);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
index f695cb1..67cc7fc 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
@@ -1,3 +1,22 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
package org.apache.flink.streaming.api.streamcomponent;
import java.io.IOException;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
index fc5978e..a6e842f 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
@@ -49,7 +49,7 @@ public class ClusterUtil {
NepheleMiniCluster exec = new NepheleMiniCluster();
exec.setMemorySize(memorySize);
exec.setNumTaskTracker(numberOfTaskTrackers);
- Client client = new Client(new InetSocketAddress("localhost", 6498), configuration);
+ Client client = new Client(new InetSocketAddress("localhost", 6498), configuration, ClusterUtil.class.getClassLoader());
if (LOG.isInfoEnabled()) {
LOG.info("Running on mini cluster");
@@ -77,7 +77,7 @@ public class ClusterUtil {
Configuration configuration = jobGraph.getJobConfiguration();
- Client client = new Client(new InetSocketAddress(IP, port), configuration);
+ Client client = new Client(new InetSocketAddress(IP, port), configuration, ClusterUtil.class.getClassLoader());
try {
client.run(jobGraph, true);
@@ -86,4 +86,4 @@ public class ClusterUtil {
}
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/pom.xml b/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
index a36946f..a848626 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
@@ -42,11 +42,6 @@ under the License.
<version>${project.version}</version>
</dependency>
- <dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>commons-lang3</artifactId>
- <version>3.1</version>
- </dependency>
</dependencies>
<build>
@@ -54,7 +49,6 @@ under the License.
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
- <version>2.4</version>
<executions>
<execution>
<goals>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-addons/flink-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/pom.xml b/flink-addons/flink-streaming/pom.xml
index 202069f..b9ad104 100644
--- a/flink-addons/flink-streaming/pom.xml
+++ b/flink-addons/flink-streaming/pom.xml
@@ -43,7 +43,7 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
- <version>${project.version}</version>
+ <version>${project.version}</version>
<type>jar</type>
</dependency>
@@ -89,37 +89,6 @@ under the License.
<type>jar</type>
</dependency>
- <dependency>
- <groupId>commons-logging</groupId>
- <artifactId>commons-logging</artifactId>
- <version>1.1.1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
-
- <dependency>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- <version>1.2.16</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
-
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <version>4.7</version>
- <type>jar</type>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <version>1.8.5</version>
- <type>jar</type>
- <scope>test</scope>
- </dependency>
</dependencies>
</project>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index ef4cd00..81d6e51 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -33,14 +33,6 @@ under the License.
<name>flink-core</name>
<packaging>jar</packaging>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>commons-lang3</artifactId>
- <version>3.1</version>
- </dependency>
- </dependencies>
<build>
<plugins>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-dist/pom.xml
----------------------------------------------------------------------
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index ac340f4..bde8d4d 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -140,12 +140,12 @@ under the License.
<goal>single</goal>
</goals>
<configuration>
- <archiverConfig>
- <!-- https://jira.codehaus.org/browse/MASSEMBLY-449 -->
- <fileMode>420</fileMode> <!-- 420(dec) = 644(oct) -->
- <directoryMode>493</directoryMode> <!-- 493(dec) = 755(oct) -->
+ <archiverConfig>
+ <!-- https://jira.codehaus.org/browse/MASSEMBLY-449 -->
+ <fileMode>420</fileMode> <!-- 420(dec) = 644(oct) -->
+ <directoryMode>493</directoryMode> <!-- 493(dec) = 755(oct) -->
<defaultDirectoryMode>493</defaultDirectoryMode>
- </archiverConfig>
+ </archiverConfig>
<archive>
<manifest>
<mainClass>org.apache.flink.yarn.Client</mainClass>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-dist/src/main/flink-bin/LICENSE
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/LICENSE b/flink-dist/src/main/flink-bin/LICENSE
index 13f25f8..15caf5a 100644
--- a/flink-dist/src/main/flink-bin/LICENSE
+++ b/flink-dist/src/main/flink-bin/LICENSE
@@ -227,6 +227,8 @@ under the Apache License (v 2.0):
- Apache Avro (http://avro.apache.org)
- Apache Hadoop (http://hadoop.apache.org)
- Apache Derby (http://db.apache.org/derby/)
+ - Apache Kafka (http://kafka.apache.org)
+ - Apache Flume (http://flume.apache.org)
- Google Guava (https://code.google.com/p/guava-libraries/)
- Netty (http://netty.io)
- Powermock (http://www.powermock.org)
@@ -236,6 +238,8 @@ under the Apache License (v 2.0):
- ScalaTest (http://www.scalatest.org)
- StartBootstrap (http://startbootstrap.com)
- CHAP Links Library Timeline (http://almende.github.io/chap-links-library/)
+ - Twitter Hosebird Client (hbc) (https://github.com/twitter/hbc)
+
-----------------------------------------------------------------------
Eclipse Public License - v 1.0
@@ -337,6 +341,21 @@ POSSIBILITY OF SUCH DAMAGE.
-----------------------------------------------------------------------
+ Mozilla Public License - v 1.1
+-----------------------------------------------------------------------
+
+The Apache Flink project depends on and/or bundles the following components
+under the Mozilla Public License (v 1.1)
+
+ - RabbitMQ (http://www.rabbitmq.com)
+ The Initial Developer of the Original Code is GoPivotal,Ltd.
+ Copyright (c) 2007-2013 GoPivotal, Inc. All Rights Reserved.
+
+You may obtain a copy of the Mozilla Public License (v 1.1) at
+http://www.mozilla.org/MPL/
+
+
+-----------------------------------------------------------------------
The Open Font License
-----------------------------------------------------------------------
@@ -344,7 +363,7 @@ The Apache Flink project packages the following fonts under the
Open Font License (OFT) - http://scripts.sil.org/OFL/
- Font Awesome (http://fortawesome.github.io/Font-Awesome/) - Created by Dave Gandy
- -> fonts in "resources/web-docs-infoserver/font-awesome/fonts"
+ -> fonts in "flink-runtime/resources/web-docs-infoserver/font-awesome/fonts"
-----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/flink-dist/src/main/flink-bin/NOTICE
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/NOTICE b/flink-dist/src/main/flink-bin/NOTICE
index 00ebeda..cf03416 100644
--- a/flink-dist/src/main/flink-bin/NOTICE
+++ b/flink-dist/src/main/flink-bin/NOTICE
@@ -89,6 +89,14 @@ projects: See https://issues.apache.org/jira/browse/HBASE-3374)
-----------------------------------------------------------------------
+ Apache Flume
+-----------------------------------------------------------------------
+
+Portions of this software were developed at
+Cloudera, Inc. (http://www.cloudera.com/).
+
+
+-----------------------------------------------------------------------
Amazon Web Services SDK for Java
-----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0163cfa1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0394053..b6b4c8f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -80,6 +80,12 @@ under the License.
<dependencies>
<dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-lang3</artifactId>
+ <version>3.1</version>
+ </dependency>
+
+ <dependency>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
<version>1.1.3</version>
@@ -324,7 +330,7 @@ under the License.
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-javadoc-plugin</artifactId>
- <version>2.9.1</version>
+ <version>2.9.1</version><!--$NO-MVN-MAN-VER$-->
<configuration>
<quiet>true</quiet>
</configuration>
[47/51] [abbrv] git commit: [streaming] Updated operator test to
avoid environment execution
Posted by se...@apache.org.
[streaming] Updated operator test to avoid environment execution
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/fc46d4c9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/fc46d4c9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/fc46d4c9
Branch: refs/heads/master
Commit: fc46d4c9123defcdb03324a464cb818d705d6cdf
Parents: 696bce0
Author: ghermann <re...@gmail.com>
Authored: Fri Aug 8 14:42:53 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:43 2014 +0200
----------------------------------------------------------------------
.../connectors/twitter/TwitterLocal.java | 2 +-
.../streaming/api/datastream/DataStream.java | 12 +-
.../environment/StreamExecutionEnvironment.java | 3 +-
.../flink/streaming/util/MockCollector.java | 41 +++
.../flink/streaming/util/MockInvokable.java | 106 ++++++
.../apache/flink/streaming/util/MockSource.java | 38 +++
.../apache/flink/streaming/api/SourceTest.java | 54 +++
.../api/invokable/operator/BatchReduceTest.java | 179 +++-------
.../api/invokable/operator/FilterTest.java | 38 +--
.../api/invokable/operator/FlatMapTest.java | 192 +----------
.../api/invokable/operator/MapTest.java | 327 +------------------
11 files changed, 323 insertions(+), 669 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
index 668647d..34ddf51 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
@@ -99,7 +99,7 @@ public class TwitterLocal implements Serializable {
return new Tuple2<String, Integer>(value, 1);
}
- }).groupReduce(new WordCountCounter(), 0);
+ }).groupBy(0).reduce(new WordCountCounter());
dataStream.addSink(new SinkFunction<Tuple2<String, Integer>>() {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 9f802f7..ab14bc6 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -301,10 +301,8 @@ public abstract class DataStream<OUT> {
* @return The transformed {@link DataStream}.
*/
public <R> SingleOutputStreamOperator<R, ?> batchReduce(GroupReduceFunction<OUT, R> reducer,
- int batchSize) {
- return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
- GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<OUT, R>(reducer,
- batchSize, batchSize));
+ long batchSize) {
+ return batchReduce(reducer, batchSize, batchSize);
}
/**
@@ -328,7 +326,7 @@ public abstract class DataStream<OUT> {
* @return The transformed {@link DataStream}.
*/
public <R> SingleOutputStreamOperator<R, ?> batchReduce(GroupReduceFunction<OUT, R> reducer,
- int batchSize, int slideSize) {
+ long batchSize, long slideSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<OUT, R>(reducer,
batchSize, slideSize));
@@ -355,9 +353,7 @@ public abstract class DataStream<OUT> {
*/
public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
long windowSize) {
- return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
- GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<OUT, R>(reducer,
- windowSize, windowSize));
+ return windowReduce(reducer, windowSize, windowSize);
}
/**
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index c357424..68e2421 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -220,7 +220,6 @@ public abstract class StreamExecutionEnvironment {
* type of the returned stream
* @return The DataStream representing the elements.
*/
- @SuppressWarnings("unchecked")
public <OUT extends Serializable> DataStreamSource<OUT> fromCollection(Collection<OUT> data) {
DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements");
@@ -233,7 +232,7 @@ public abstract class StreamExecutionEnvironment {
jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(
new FromElementsFunction<OUT>(data)), new ObjectTypeWrapper<OUT, Tuple, OUT>(
- (OUT) data.toArray()[0], null, (OUT) data.toArray()[0]), "source",
+ data.iterator().next(), null, data.iterator().next()), "source",
SerializationUtils.serialize(function), 1);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize collection");
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockCollector.java
new file mode 100644
index 0000000..e200d70
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockCollector.java
@@ -0,0 +1,41 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.util;
+
+import java.util.Collection;
+
+import org.apache.flink.util.Collector;
+
+public class MockCollector<T> implements Collector<T> {
+ private Collection<T> outputs;
+
+ public MockCollector(Collection<T> outputs) {
+ this.outputs = outputs;
+ }
+
+ @Override
+ public void collect(T record) {
+ outputs.add(record);
+ }
+
+ @Override
+ public void close() {
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockInvokable.java
new file mode 100644
index 0000000..91c48e1
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockInvokable.java
@@ -0,0 +1,106 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.util;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+
+public class MockInvokable<IN, OUT> {
+ private Collection<IN> inputs;
+ private List<OUT> outputs;
+
+ private Collector<OUT> collector;
+ private StreamRecordSerializer<IN> inDeserializer;
+ private MutableObjectIterator<StreamRecord<IN>> iterator;
+
+ public MockInvokable(Collection<IN> inputs) {
+ this.inputs = inputs;
+ if (inputs.isEmpty()) {
+ throw new RuntimeException("Inputs must not be empty");
+ }
+
+ TypeInformation<IN> inTypeInfo = TypeExtractor.getForObject(inputs.iterator().next());
+ inDeserializer = new StreamRecordSerializer<IN>(inTypeInfo);
+
+ iterator = new MockInputIterator();
+ outputs = new ArrayList<OUT>();
+ collector = new MockCollector<OUT>(outputs);
+ }
+
+
+ private class MockInputIterator implements MutableObjectIterator<StreamRecord<IN>> {
+ Iterator<IN> listIterator;
+
+ public MockInputIterator() {
+ listIterator = inputs.iterator();
+ }
+
+ @Override
+ public StreamRecord<IN> next(StreamRecord<IN> reuse) throws IOException {
+ if (listIterator.hasNext()) {
+ reuse.setObject(listIterator.next());
+ } else {
+ reuse = null;
+ }
+ return reuse;
+ }
+ }
+
+ public List<OUT> getOutputs() {
+ return outputs;
+ }
+
+ public Collector<OUT> getCollector() {
+ return collector;
+ }
+
+ public StreamRecordSerializer<IN> getInDeserializer() {
+ return inDeserializer;
+ }
+
+ public MutableObjectIterator<StreamRecord<IN>> getIterator() {
+ return iterator;
+ }
+
+ public static <IN, OUT> List<OUT> createAndExecute(UserTaskInvokable<IN, OUT> invokable, List<IN> inputs) {
+ MockInvokable<IN, OUT> mock = new MockInvokable<IN, OUT>(inputs);
+ invokable.initialize(mock.getCollector(), mock.getIterator(), mock.getInDeserializer(), false);
+ try {
+ invokable.open(null);
+ invokable.invoke();
+ } catch (Exception e) {
+ throw new RuntimeException("Cannot invoke invokable.", e);
+ }
+
+ return mock.getOutputs();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockSource.java
new file mode 100644
index 0000000..cbace31
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/MockSource.java
@@ -0,0 +1,38 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.streaming.api.function.source.SourceFunction;
+
+public class MockSource<T> {
+
+ public static <T> List<T> createAndExecute(SourceFunction<T> source) {
+ List<T> outputs = new ArrayList<T>();
+ try {
+ source.invoke(new MockCollector<T>(outputs));
+ } catch (Exception e) {
+ throw new RuntimeException("Cannot invoke source.", e);
+ }
+ return outputs;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/SourceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/SourceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/SourceTest.java
new file mode 100644
index 0000000..750d846
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/SourceTest.java
@@ -0,0 +1,54 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.streaming.api.function.source.FromElementsFunction;
+import org.apache.flink.streaming.api.function.source.GenSequenceFunction;
+import org.apache.flink.streaming.util.MockSource;
+import org.junit.Test;
+
+public class SourceTest {
+
+ @Test
+ public void fromElementsTest() {
+ List<Integer> expectedList = Arrays.asList(1, 2, 3);
+ List<Integer> actualList = MockSource.createAndExecute(new FromElementsFunction<Integer>(1, 2, 3));
+ assertEquals(expectedList, actualList);
+ }
+
+ @Test
+ public void fromCollectionTest() {
+ List<Integer> expectedList = Arrays.asList(1, 2, 3);
+ List<Integer> actualList = MockSource.createAndExecute(new FromElementsFunction<Integer>(Arrays.asList(1, 2, 3)));
+ assertEquals(expectedList, actualList);
+ }
+
+ @Test
+ public void genSequenceTest() {
+ List<Long> expectedList = Arrays.asList(1L, 2L, 3L);
+ List<Long> actualList = MockSource.createAndExecute(new GenSequenceFunction(1, 3));
+ assertEquals(expectedList, actualList);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
index bf7621e..63ba627 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -22,176 +22,79 @@ package org.apache.flink.streaming.api.invokable.operator;
import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
-import java.util.SortedSet;
-import java.util.TreeSet;
import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.streaming.util.LogUtils;
+import org.apache.flink.streaming.util.MockInvokable;
import org.apache.flink.util.Collector;
-import org.apache.log4j.Level;
import org.junit.Test;
-public class BatchReduceTest {
-
- private static ArrayList<Double> avgs = new ArrayList<Double>();
- private static final int BATCH_SIZE = 5;
- private static final int PARALLELISM = 1;
- private static final long MEMORYSIZE = 32;
-
- public static final class MyBatchReduce implements GroupReduceFunction<Double, Double> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void reduce(Iterable<Double> values, Collector<Double> out) throws Exception {
-
- Double sum = 0.;
- Double count = 0.;
- for (Double value : values) {
- sum += value;
- count++;
- }
- if (count > 0) {
- out.collect(new Double(sum / count));
- }
- }
- }
- public static final class MySink implements SinkFunction<Double> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Double tuple) {
- avgs.add(tuple);
- }
-
- }
-
- public static final class MySource implements SourceFunction<Double> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Collector<Double> collector) {
- for (Double i = 1.; i <= 100; i++) {
- collector.collect(new Double(i));
- }
- }
- }
+public class BatchReduceTest {
- public static final class MySlidingBatchReduce implements RichFunction,
- GroupReduceFunction<Long, String> {
+ public static final class MySlidingBatchReduce implements GroupReduceFunction<Integer, String> {
private static final long serialVersionUID = 1L;
- double startTime;
-
@Override
- public void reduce(Iterable<Long> values, Collector<String> out) throws Exception {
- for (Long value : values) {
+ public void reduce(Iterable<Integer> values, Collector<String> out) throws Exception {
+ for (Integer value : values) {
out.collect(value.toString());
}
out.collect(END_OF_BATCH);
}
+ }
- @Override
- public void open(Configuration parameters) throws Exception {
- startTime = (double) System.currentTimeMillis() / 1000;
- }
-
- @Override
- public void close() throws Exception {
- }
+ private final static String END_OF_BATCH = "end of batch";
+ private final static int SLIDING_BATCH_SIZE = 3;
+ private final static int SLIDE_SIZE = 2;
- @Override
- public RuntimeContext getRuntimeContext() {
- return null;
- }
+ @Test
+ public void slidingBatchReduceTest() {
+ BatchReduceInvokable<Integer, String> invokable = new BatchReduceInvokable<Integer, String>(
+ new MySlidingBatchReduce(), SLIDING_BATCH_SIZE, SLIDE_SIZE);
- @Override
- public void setRuntimeContext(RuntimeContext t) {
- // TODO Auto-generated method stub
+ List<String> expected = Arrays.asList("1", "2", "3", END_OF_BATCH, "3", "4", "5",
+ END_OF_BATCH, "5", "6", "7", END_OF_BATCH);
+ List<String> actual = MockInvokable.createAndExecute(invokable,
+ Arrays.asList(1, 2, 3, 4, 5, 6, 7));
- }
+ assertEquals(expected, actual);
}
- private static List<SortedSet<String>> sink = new ArrayList<SortedSet<String>>();
- private static final String END_OF_BATCH = "end of batch";
-
- public static final class MySlidingSink implements SinkFunction<String> {
-
+ public static final class MyBatchReduce implements GroupReduceFunction<Double, Double> {
private static final long serialVersionUID = 1L;
- SortedSet<String> currentSet = new TreeSet<String>();
-
@Override
- public void invoke(String string) {
- if (string.equals(END_OF_BATCH)) {
- sink.add(currentSet);
- currentSet = new TreeSet<String>();
- } else {
- currentSet.add(string);
+ public void reduce(Iterable<Double> values, Collector<Double> out) throws Exception {
+
+ Double sum = 0.;
+ Double count = 0.;
+ for (Double value : values) {
+ sum += value;
+ count++;
+ }
+ if (count > 0) {
+ out.collect(new Double(sum / count));
}
}
}
-
- private final static int SLIDING_BATCH_SIZE = 9;
- private final static int SLIDE_SIZE = 6;
- private static final int SEQUENCE_SIZE = 30;
- private LocalStreamEnvironment env;
- private void slidingStream() {
- env.generateSequence(1, SEQUENCE_SIZE)
- .batchReduce(new MySlidingBatchReduce(), SLIDING_BATCH_SIZE, SLIDE_SIZE)
- .addSink(new MySlidingSink());
- }
-
- private void slidingTest() {
- int firstInBatch = 1;
+ private static final int BATCH_SIZE = 5;
- for (SortedSet<String> set : sink) {
- int to = Math.min(firstInBatch + SLIDING_BATCH_SIZE - 1, SEQUENCE_SIZE);
- assertEquals(getExpectedSet(to), set);
- firstInBatch += SLIDE_SIZE;
- }
- }
-
- private void nonSlidingStream() {
- env.addSource(new MySource()).batchReduce(new MyBatchReduce(), BATCH_SIZE)
- .addSink(new MySink());
- }
-
- private void nonSlidingTest() {
- for (int i = 0; i < avgs.size(); i++) {
- assertEquals(3.0 + i * BATCH_SIZE, avgs.get(i), 0);
- }
- }
-
@Test
- public void test() {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
-
- env = StreamExecutionEnvironment.createLocalEnvironment(PARALLELISM);
-
- slidingStream();
- nonSlidingStream();
+ public void nonSlidingBatchReduceTest() {
+ List<Double> inputs = new ArrayList<Double>();
+ for (Double i = 1.; i <= 100; i++) {
+ inputs.add(i);
+ }
- env.executeTest(MEMORYSIZE);
-
- slidingTest();
- nonSlidingTest();
- }
+ BatchReduceInvokable<Double, Double> invokable = new BatchReduceInvokable<Double, Double>(new MyBatchReduce(), BATCH_SIZE, BATCH_SIZE);
+
+ List<Double> avgs = MockInvokable.createAndExecute(invokable, inputs);
- private SortedSet<String> getExpectedSet(int to) {
- SortedSet<String> expectedSet = new TreeSet<String>();
- for (int i = to; i > to - SLIDING_BATCH_SIZE; i--) {
- expectedSet.add(Integer.toString(i));
+ for (int i = 0; i < avgs.size(); i++) {
+ assertEquals(3.0 + i * BATCH_SIZE, avgs.get(i), 0);
}
- return expectedSet;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
index ec625e9..152f992 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FilterTest.java
@@ -19,32 +19,19 @@
package org.apache.flink.streaming.api.invokable.operator;
+import static org.junit.Assert.assertEquals;
+
import java.io.Serializable;
-import java.util.HashSet;
-import java.util.Set;
+import java.util.Arrays;
+import java.util.List;
import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.util.LogUtils;
-import org.apache.log4j.Level;
-import org.junit.Assert;
+import org.apache.flink.streaming.util.MockInvokable;
import org.junit.Test;
public class FilterTest implements Serializable {
private static final long serialVersionUID = 1L;
- private static Set<Integer> set = new HashSet<Integer>();
-
- private static class MySink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer value) {
- set.add(value);
- }
- }
-
static class MyFilter implements FilterFunction<Integer> {
private static final long serialVersionUID = 1L;
@@ -54,16 +41,13 @@ public class FilterTest implements Serializable {
}
}
- @Test
+ @Test
public void test() {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
- env.fromElements(1, 2, 3, 4, 5, 6, 7).filter(new MyFilter()).addSink(new MySink());
-
- env.execute();
+ FilterInvokable<Integer> invokable = new FilterInvokable<Integer>(new MyFilter());
- Assert.assertArrayEquals(new Integer[] { 2, 4, 6 }, set.toArray());
+ List<Integer> expected = Arrays.asList(2, 4, 6);
+ List<Integer> actual = MockInvokable.createAndExecute(invokable, Arrays.asList(1, 2, 3, 4, 5, 6, 7));
+
+ assertEquals(expected, actual);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
index 06f8447..fe367d3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
@@ -20,19 +20,13 @@
package org.apache.flink.streaming.api.invokable.operator;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import java.util.HashSet;
-import java.util.Set;
+import java.util.Arrays;
+import java.util.List;
import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.util.LogUtils;
+import org.apache.flink.streaming.util.MockInvokable;
import org.apache.flink.util.Collector;
-import org.apache.log4j.Level;
import org.junit.Test;
public class FlatMapTest {
@@ -43,180 +37,20 @@ public class FlatMapTest {
@Override
public void flatMap(Integer value, Collector<Integer> out) throws Exception {
- out.collect(value * value);
-
- }
-
- }
-
- public static final class ParallelFlatMap implements FlatMapFunction<Integer, Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void flatMap(Integer value, Collector<Integer> out) throws Exception {
- numberOfElements++;
-
- }
-
- }
-
- public static final class GenerateSequenceFlatMap implements FlatMapFunction<Long, Long> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void flatMap(Long value, Collector<Long> out) throws Exception {
- out.collect(value * value);
-
- }
-
- }
-
- public static final class MySink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- result.add(tuple);
- }
-
- }
-
- public static final class FromElementsSink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- fromElementsResult.add(tuple);
- }
-
- }
-
- public static final class FromCollectionSink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- fromCollectionResult.add(tuple);
- }
-
- }
-
- public static final class GenerateSequenceSink implements SinkFunction<Long> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Long tuple) {
- generateSequenceResult.add(tuple);
- }
-
- }
-
- 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);
+ if (value % 2 == 0) {
+ out.collect(value);
+ out.collect(value * value);
}
}
}
- private static final int PARALLELISM = 1;
- private static final long MEMORYSIZE = 32;
-
- private static int numberOfElements = 0;
- private static Set<Integer> expected = new HashSet<Integer>();
- private static Set<Integer> result = new HashSet<Integer>();
- private static Set<Integer> fromElementsExpected = new HashSet<Integer>();
- private static Set<Integer> fromElementsResult = new HashSet<Integer>();
- private static Set<Integer> fromCollectionSet = new HashSet<Integer>();
- private static Set<Integer> sequenceExpected = new HashSet<Integer>();
- private static Set<Long> sequenceLongExpected = new HashSet<Long>();
- private static Set<Integer> fromCollectionResult = new HashSet<Integer>();
- private static Set<Long> generateSequenceResult = new HashSet<Long>();
-
@Test
- public void test() throws Exception {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
-
- LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(PARALLELISM);
- // flatmapTest
-
- fillFromCollectionSet();
-
- @SuppressWarnings("unused")
- DataStream<Integer> dataStream = env.fromCollection(fromCollectionSet)
- .flatMap(new MyFlatMap()).addSink(new MySink());
-
- fillExpectedList();
-
- // parallelShuffleconnectTest
- fillFromCollectionSet();
-
- DataStream<Integer> source = env.fromCollection(fromCollectionSet);
- @SuppressWarnings("unused")
- DataStream<Integer> map = source.flatMap(new ParallelFlatMap()).addSink(
- new MySink());
- @SuppressWarnings("unused")
- DataStream<Integer> map2 = source.flatMap(new ParallelFlatMap()).addSink(
- new MySink());
-
- // fromElementsTest
- DataStream<Integer> fromElementsMap = env.fromElements(2, 5, 9).flatMap(
- new MyFlatMap());
- @SuppressWarnings("unused")
- DataStream<Integer> sink = fromElementsMap.addSink(new FromElementsSink());
-
- fillFromElementsExpected();
-
- // fromCollectionTest
- fillFromCollectionSet();
-
- DataStream<Integer> fromCollectionMap = env.fromCollection(fromCollectionSet)
- .flatMap(new MyFlatMap());
- @SuppressWarnings("unused")
- DataStream<Integer> fromCollectionSink = fromCollectionMap
- .addSink(new FromCollectionSink());
-
- // generateSequenceTest
- fillSequenceSet();
-
- DataStream<Long> generateSequenceMap = env.generateSequence(0, 9).flatMap(
- new GenerateSequenceFlatMap());
- @SuppressWarnings("unused")
- DataStream<Long> generateSequenceSink = generateSequenceMap
- .addSink(new GenerateSequenceSink());
-
- fillLongSequenceSet();
-
- env.executeTest(MEMORYSIZE);
-
- assertTrue(expected.equals(result));
- assertEquals(20, numberOfElements);
- assertEquals(fromElementsExpected, fromElementsResult);
- assertEquals(sequenceExpected, fromCollectionResult);
- assertEquals(sequenceLongExpected, generateSequenceResult);
+ public void flatMapTest() {
+ FlatMapInvokable<Integer, Integer> invokable = new FlatMapInvokable<Integer, Integer>(new MyFlatMap());
+
+ List<Integer> expected = Arrays.asList(2, 4, 4, 16, 6, 36, 8, 64);
+ List<Integer> actual = MockInvokable.createAndExecute(invokable, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8));
+
+ assertEquals(expected, actual);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fc46d4c9/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
index 55624d6..e3c7cb7 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
@@ -20,333 +20,32 @@
package org.apache.flink.streaming.api.invokable.operator;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import java.util.ArrayList;
-import java.util.HashSet;
+import java.util.Arrays;
import java.util.List;
-import java.util.Set;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.streaming.util.LogUtils;
-import org.apache.flink.util.Collector;
-import org.apache.log4j.Level;
+import org.apache.flink.streaming.util.MockInvokable;
import org.junit.Test;
public class MapTest {
- public static final class MySource implements SourceFunction<Integer> {
+ private static class Map implements MapFunction<Integer, String> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Collector<Integer> collector) throws Exception {
- for (int i = 0; i < 10; i++) {
- collector.collect(i);
- }
+ public String map(Integer value) throws Exception {
+ return "+" + (value + 1);
}
}
-
- public static final class MySource1 implements SourceFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Collector<Integer> collector) throws Exception {
- for (int i = 0; i < 5; i++) {
- collector.collect(i);
- }
- }
- }
-
- public static final class MySource2 implements SourceFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Collector<Integer> collector) throws Exception {
- for (int i = 5; i < 10; i++) {
- collector.collect(i);
- }
- }
- }
-
- public static final class MySource3 implements SourceFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Collector<Integer> collector) throws Exception {
- for (int i = 10; i < 15; i++) {
- collector.collect(new Integer(i));
- }
- }
- }
-
- public static final class MyMap implements MapFunction<Integer, Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Integer map(Integer value) throws Exception {
- map++;
- return value * value;
- }
- }
-
- public static final class MySingleJoinMap implements MapFunction<Integer, Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Integer map(Integer value) throws Exception {
- singleJoinSetResult.add(value);
- return value;
- }
- }
-
- public static final class MyMultipleJoinMap implements MapFunction<Integer, Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Integer map(Integer value) throws Exception {
- multipleJoinSetResult.add(value);
- return value;
- }
- }
-
- public static final class MyFieldsMap implements MapFunction<Integer, Integer> {
- private static final long serialVersionUID = 1L;
-
- private int counter = 0;
-
- @Override
- public Integer map(Integer value) throws Exception {
- counter++;
- if (counter == MAXSOURCE)
- allInOne = true;
- return value * value;
- }
- }
-
- public static final class MyDiffFieldsMap implements MapFunction<Integer, Integer> {
- private static final long serialVersionUID = 1L;
-
- private int counter = 0;
-
- @Override
- public Integer map(Integer value) throws Exception {
- counter++;
- if (counter > 3)
- threeInAll = false;
- return value * value;
- }
- }
-
- public static final class MySink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- result.add(tuple);
- }
- }
-
- public static final class MyBroadcastSink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- broadcastResult++;
- }
- }
-
- public static final class MyShufflesSink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- shuffleResult++;
- }
- }
-
- public static final class MyFieldsSink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- fieldsResult++;
- }
- }
-
- public static final class MyDiffFieldsSink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- diffFieldsResult++;
- }
- }
-
- public static final class MyGraphSink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- graphResult++;
- }
- }
-
- public static final class JoinSink implements SinkFunction<Integer> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Integer tuple) {
- }
- }
-
- private static Set<Integer> expected = new HashSet<Integer>();
- private static Set<Integer> result = new HashSet<Integer>();
- private static int broadcastResult = 0;
- private static int shuffleResult = 0;
- @SuppressWarnings("unused")
- private static int fieldsResult = 0;
- private static int diffFieldsResult = 0;
- @SuppressWarnings("unused")
- private static int graphResult = 0;
- @SuppressWarnings("unused")
- private static int map = 0;
- @SuppressWarnings("unused")
- private static final int PARALLELISM = 1;
- private static final long MEMORYSIZE = 32;
- private static final int MAXSOURCE = 10;
- private static boolean allInOne = false;
- private static boolean threeInAll = true;
- private static Set<Integer> fromCollectionSet = new HashSet<Integer>();
- private static List<Integer> fromCollectionFields = new ArrayList<Integer>();
- private static Set<Integer> fromCollectionDiffFieldsSet = new HashSet<Integer>();
- private static Set<Integer> singleJoinSetExpected = new HashSet<Integer>();
- private static Set<Integer> multipleJoinSetExpected = new HashSet<Integer>();
- private static Set<Integer> singleJoinSetResult = new HashSet<Integer>();
- private static Set<Integer> multipleJoinSetResult = new HashSet<Integer>();
-
- 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 {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
- // mapTest
- LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(3);
-
- fillFromCollectionSet();
-
- @SuppressWarnings("unused")
- DataStream<Integer> dataStream = env.fromCollection(fromCollectionSet).map(new MyMap())
- .addSink(new MySink());
-
- fillExpectedList();
-
- // broadcastSinkTest
- fillFromCollectionSet();
-
- @SuppressWarnings("unused")
- DataStream<Integer> dataStream1 = env.fromCollection(fromCollectionSet).broadcast()
- .map(new MyMap()).addSink(new MyBroadcastSink());
-
- // shuffleSinkTest
- fillFromCollectionSet();
-
- @SuppressWarnings("unused")
- DataStream<Integer> dataStream2 = env.fromCollection(fromCollectionSet).map(new MyMap())
- .setParallelism(3).addSink(new MyShufflesSink());
-
- // fieldsMapTest
- fillFromCollectionFieldsSet();
-
- @SuppressWarnings("unused")
- DataStream<Integer> dataStream3 = env.fromCollection(fromCollectionFields).partitionBy(0)
- .map(new MyFieldsMap()).addSink(new MyFieldsSink());
-
- // diffFieldsMapTest
- fillFromCollectionDiffFieldsSet();
-
- @SuppressWarnings("unused")
- DataStream<Integer> dataStream4 = env.fromCollection(fromCollectionDiffFieldsSet)
- .partitionBy(0).map(new MyDiffFieldsMap()).addSink(new MyDiffFieldsSink());
-
- // singleConnectWithTest
- DataStream<Integer> source1 = env.addSource(new MySource1(), 1);
-
- @SuppressWarnings({ "unused", "unchecked" })
- DataStream<Integer> source2 = env.addSource(new MySource2(), 1).merge(source1)
- .partitionBy(0).map(new MySingleJoinMap()).setParallelism(1)
- .addSink(new JoinSink());
-
- fillSingleJoinSet();
-
- // multipleConnectWithTest
- DataStream<Integer> source3 = env.addSource(new MySource1(), 1);
-
- DataStream<Integer> source4 = env.addSource(new MySource2(), 1);
-
- @SuppressWarnings({ "unused", "unchecked" })
- DataStream<Integer> source5 = env.addSource(new MySource3(), 1)
- .merge(source3, source4).partitionBy(0).map(new MyMultipleJoinMap())
- .setParallelism(1).addSink(new JoinSink());
-
- env.executeTest(MEMORYSIZE);
-
- fillMultipleJoinSet();
-
- assertTrue(expected.equals(result));
- assertEquals(30, broadcastResult);
- assertEquals(10, shuffleResult);
- assertTrue(allInOne);
- assertTrue(threeInAll);
- assertEquals(9, diffFieldsResult);
- assertEquals(singleJoinSetExpected, singleJoinSetResult);
- assertEquals(multipleJoinSetExpected, multipleJoinSetResult);
-
+ public void mapInvokableTest() {
+ MapInvokable<Integer, String> invokable = new MapInvokable<Integer, String>(new Map());
+
+ List<String> expectedList = Arrays.asList("+2", "+3", "+4");
+ List<String> actualList = MockInvokable.createAndExecute(invokable, Arrays.asList(1, 2, 3));
+
+ assertEquals(expectedList, actualList);
}
-
}
[37/51] [abbrv] git commit: [streaming] Directed emit API updated to
use split and select
Posted by se...@apache.org.
[streaming] Directed emit API updated to use split and select
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/910f74d7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/910f74d7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/910f74d7
Branch: refs/heads/master
Commit: 910f74d7bbef96e2a258601666daca326cfcad62
Parents: 776bd3f
Author: gyfora <gy...@gmail.com>
Authored: Sun Aug 3 19:20:09 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:50 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 88 ++++++++++++--------
.../flink/streaming/api/NamedDataStream.java | 33 ++++++++
.../flink/streaming/api/SplitDataStream.java | 48 +++++++++++
.../api/collector/DirectedOutputTest.java | 7 +-
4 files changed, 138 insertions(+), 38 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/910f74d7/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index f17dd1b..c648ab2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -124,7 +124,15 @@ public class DataStream<T> {
this.iterationID = dataStream.iterationID;
this.jobGraphBuilder = dataStream.jobGraphBuilder;
}
-
+
+ /**
+ * Creates a copy of the DataStream
+ *
+ * @return The copy
+ */
+ protected DataStream<T> copy() {
+ return new DataStream<T>(this);
+ }
/**
* Partitioning strategy on the stream.
@@ -197,7 +205,7 @@ public class DataStream<T> {
jobGraphBuilder.setParallelism(id, degreeOfParallelism);
- return new DataStream<T>(this);
+ return this;
}
/**
@@ -218,7 +226,7 @@ public class DataStream<T> {
* The name to set
* @return The named DataStream.
*/
- public DataStream<T> name(String name) {
+ protected DataStream<T> name(String name) {
// TODO copy DataStream?
if (name == "") {
throw new IllegalArgumentException("User defined name must not be empty string");
@@ -241,7 +249,7 @@ public class DataStream<T> {
* @return The connected DataStream.
*/
public DataStream<T> connectWith(DataStream<T>... streams) {
- DataStream<T> returnStream = new DataStream<T>(this);
+ DataStream<T> returnStream = this.copy();
for (DataStream<T> stream : streams) {
addConnection(returnStream, stream);
@@ -270,16 +278,18 @@ public class DataStream<T> {
*
* @param outputSelector
* The user defined OutputSelector for directing the tuples.
- * @return The directed DataStream.
+ * @return The {@link SplitDataStream}
*/
- public DataStream<T> directTo(OutputSelector<T> outputSelector) {
+ public SplitDataStream<T> split(OutputSelector<T> outputSelector) {
try {
- jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
+ for (String id : connectIDs) {
+ jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
+ }
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize OutputSelector");
}
- return this;
+ return new SplitDataStream<T>(this);
}
/**
@@ -340,7 +350,7 @@ public class DataStream<T> {
}
private DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
- DataStream<T> returnStream = new DataStream<T>(this);
+ DataStream<T> returnStream = this.copy();
for (int i = 0; i < returnStream.partitioners.size(); i++) {
returnStream.partitioners.set(i, partitioner);
@@ -383,8 +393,7 @@ public class DataStream<T> {
*/
public <T2, R> DataStream<R> coMapWith(CoMapFunction<T, T2, R> coMapper,
DataStream<T2> otherStream) {
- return addCoFunction("coMap", new DataStream<T>(this), new DataStream<T2>(otherStream),
- coMapper,
+ return addCoFunction("coMap", this.copy(), otherStream.copy(), coMapper,
new FunctionTypeWrapper<T, T2, R>(coMapper, CoMapFunction.class, 0, 1, 2),
new CoMapInvokable<T, T2, R>(coMapper));
}
@@ -490,7 +499,7 @@ public class DataStream<T> {
final AbstractRichFunction function, TypeSerializerWrapper<T, Tuple, R> typeWrapper,
UserTaskInvokable<T, R> functionInvokable) {
- DataStream<T> inputStream = new DataStream<T>(this);
+ DataStream<T> inputStream = this.copy();
StreamOperator<T, R> returnStream = new StreamOperator<T, R>(environment, functionName);
try {
@@ -507,6 +516,10 @@ public class DataStream<T> {
inputStream.iterationflag = false;
}
+ if (inputStream instanceof NamedDataStream) {
+ returnStream.name(inputStream.userDefinedName);
+ }
+
return returnStream;
}
@@ -527,6 +540,17 @@ public class DataStream<T> {
connectGraph(inputStream1, returnStream.getId(), 1);
connectGraph(inputStream2, returnStream.getId(), 2);
+ if ((inputStream1 instanceof NamedDataStream) && (inputStream2 instanceof NamedDataStream)) {
+ throw new RuntimeException("An operator cannot have two names");
+ } else {
+ if (inputStream1 instanceof NamedDataStream) {
+ returnStream.name(inputStream1.userDefinedName);
+ }
+
+ if (inputStream2 instanceof NamedDataStream) {
+ returnStream.name(inputStream2.userDefinedName);
+ }
+ }
// TODO consider iteration
return returnStream;
@@ -564,7 +588,7 @@ public class DataStream<T> {
* @return The closed DataStream.
*/
public DataStream<T> addSink(SinkFunction<T> sinkFunction) {
- return addSink(new DataStream<T>(this), sinkFunction);
+ return addSink(this.copy(), sinkFunction);
}
/**
@@ -575,7 +599,7 @@ public class DataStream<T> {
* @return The closed DataStream.
*/
public DataStream<T> print() {
- DataStream<T> inputStream = new DataStream<T>(this);
+ DataStream<T> inputStream = this.copy();
PrintSinkFunction<T> printFunction = new PrintSinkFunction<T>();
DataStream<T> returnStream = addSink(inputStream, printFunction, null);
@@ -603,6 +627,10 @@ public class DataStream<T> {
inputStream.connectGraph(inputStream, returnStream.getId(), 0);
+ if (this.copy() instanceof NamedDataStream) {
+ returnStream.name(inputStream.userDefinedName);
+ }
+
return returnStream;
}
@@ -617,8 +645,7 @@ public class DataStream<T> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path) {
- writeAsText(this, path, new WriteFormatAsText<T>(), 1, null);
- return new DataStream<T>(this);
+ return writeAsText(this, path, new WriteFormatAsText<T>(), 1, null);
}
/**
@@ -635,8 +662,7 @@ public class DataStream<T> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path, long millis) {
- writeAsText(this, path, new WriteFormatAsText<T>(), millis, null);
- return new DataStream<T>(this);
+ return writeAsText(this, path, new WriteFormatAsText<T>(), millis, null);
}
/**
@@ -654,8 +680,7 @@ public class DataStream<T> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path, int batchSize) {
- writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, null);
- return new DataStream<T>(this);
+ return writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, null);
}
/**
@@ -677,8 +702,7 @@ public class DataStream<T> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path, long millis, T endTuple) {
- writeAsText(this, path, new WriteFormatAsText<T>(), millis, endTuple);
- return new DataStream<T>(this);
+ return writeAsText(this, path, new WriteFormatAsText<T>(), millis, endTuple);
}
/**
@@ -701,8 +725,7 @@ public class DataStream<T> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path, int batchSize, T endTuple) {
- writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, endTuple);
- return new DataStream<T>(this);
+ return writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, endTuple);
}
/**
@@ -771,8 +794,7 @@ public class DataStream<T> {
* @return The closed DataStream
*/
public DataStream<T> writeAsCsv(String path) {
- writeAsCsv(this, path, new WriteFormatAsCsv<T>(), 1, null);
- return new DataStream<T>(this);
+ return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), 1, null);
}
/**
@@ -789,8 +811,7 @@ public class DataStream<T> {
* @return The closed DataStream
*/
public DataStream<T> writeAsCsv(String path, long millis) {
- writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, null);
- return new DataStream<T>(this);
+ return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, null);
}
/**
@@ -808,8 +829,7 @@ public class DataStream<T> {
* @return The closed DataStream
*/
public DataStream<T> writeAsCsv(String path, int batchSize) {
- writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, null);
- return new DataStream<T>(this);
+ return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, null);
}
/**
@@ -831,8 +851,7 @@ public class DataStream<T> {
* @return The closed DataStream
*/
public DataStream<T> writeAsCsv(String path, long millis, T endTuple) {
- writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, endTuple);
- return new DataStream<T>(this);
+ return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, endTuple);
}
/**
@@ -856,8 +875,7 @@ public class DataStream<T> {
*/
public DataStream<T> writeAsCsv(String path, int batchSize, T endTuple) {
setMutability(false);
- writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, endTuple);
- return new DataStream<T>(this);
+ return writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, endTuple);
}
/**
@@ -942,6 +960,6 @@ public class DataStream<T> {
jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
degreeOfParallelism);
- return new DataStream<T>(this);
+ return this.copy();
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/910f74d7/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/NamedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/NamedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/NamedDataStream.java
new file mode 100755
index 0000000..1edfa6f
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/NamedDataStream.java
@@ -0,0 +1,33 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api;
+
+public class NamedDataStream<T> extends DataStream<T> {
+
+ protected NamedDataStream(DataStream<T> dataStream) {
+ super(dataStream);
+ }
+
+ @Override
+ protected DataStream<T> copy() {
+ return new NamedDataStream<T>(this);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/910f74d7/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
new file mode 100755
index 0000000..b4bbe52
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
@@ -0,0 +1,48 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api;
+
+public class SplitDataStream<T> extends DataStream<T> {
+
+ protected SplitDataStream(DataStream<T> dataStream) {
+ super(dataStream);
+ }
+
+ /**
+ * Sets the output name for which the vertex will receive tuples from the
+ * preceding Directed stream
+ *
+ * @param outputName
+ * The output name for which the operator will receive the input.
+ * @return Returns the modified DataStream
+ */
+ public NamedDataStream<T> select(String outputName) {
+
+ userDefinedName = outputName;
+
+ return new NamedDataStream<T>(this);
+ }
+
+ @Override
+ protected DataStream<T> copy() {
+ return new SplitDataStream<T>(this);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/910f74d7/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index e0da783..074992b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -27,6 +27,7 @@ import java.util.HashSet;
import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.SplitDataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
@@ -95,9 +96,9 @@ public class DirectedOutputTest {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
- DataStream<Long> s = env.generateSequence(1, 6).directTo(new MySelector());
- DataStream<Long> ds1 = s.map(new PlusTwo()).name("ds1").addSink(new EvenSink());
- DataStream<Long> ds2 = s.map(new PlusTwo()).name("ds2").addSink(new OddSink());
+ SplitDataStream<Long> s = env.generateSequence(1, 6).split(new MySelector());
+ DataStream<Long> ds1 = s.select("ds1").shuffle().map(new PlusTwo()).addSink(new EvenSink());
+ DataStream<Long> ds2 = s.select("ds2").map(new PlusTwo()).addSink(new OddSink());
DataStream<Long> ds3 = s.map(new PlusTwo()).addSink(new OddSink());
env.execute();
[12/51] [abbrv] git commit: [streaming] Fixed multiple input
CoFunction
Posted by se...@apache.org.
[streaming] Fixed multiple input CoFunction
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/b92ce014
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/b92ce014
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/b92ce014
Branch: refs/heads/master
Commit: b92ce014fa178223119eab3b1f950d0c83630602
Parents: f436690
Author: ghermann <re...@gmail.com>
Authored: Tue Jul 22 15:07:30 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:19:17 2014 +0200
----------------------------------------------------------------------
.../flink/streaming/api/JobGraphBuilder.java | 9 ++-
.../api/streamcomponent/CoStreamTask.java | 6 +-
.../api/invokable/operator/CoMapTest.java | 81 ++++++++++++++------
3 files changed, 69 insertions(+), 27 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b92ce014/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index 022fcd8..73f8d3a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -754,17 +754,20 @@ public class JobGraphBuilder {
for (String componentName : outEdgeList.keySet()) {
createVertex(componentName);
}
- int inputNumber = 0;
+
for (String upStreamComponentName : outEdgeList.keySet()) {
-
int i = 0;
+
ArrayList<Integer> outEdgeTypeList = outEdgeType.get(upStreamComponentName);
for (String downStreamComponentName : outEdgeList.get(upStreamComponentName)) {
Configuration downStreamComponentConfig = components.get(downStreamComponentName)
.getConfiguration();
+
+ int inputNumber = downStreamComponentConfig.getInteger("numberOfInputs", 0);
downStreamComponentConfig.setInteger("inputType_" + inputNumber++, outEdgeTypeList.get(i));
-
+ downStreamComponentConfig.setInteger("numberOfInputs", inputNumber);
+
connect(upStreamComponentName, downStreamComponentName,
connectionTypes.get(upStreamComponentName).get(i));
i++;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b92ce014/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
index 4c1cf42..204bcfe 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -168,8 +168,10 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
if (inputList.size() == 1) {
return inputList.get(0);
} else if (inputList.size() > 1) {
- return new MutableUnionRecordReader<IOReadableWritable>(
- (MutableRecordReader<IOReadableWritable>[]) inputList.toArray());
+ MutableRecordReader<IOReadableWritable>[] inputArray = inputList
+ .toArray(new MutableRecordReader[inputList.size()]);
+
+ return new MutableUnionRecordReader<IOReadableWritable>(inputArray);
}
return null;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b92ce014/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
index 00ae3b9..b2fd3cf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
@@ -28,17 +28,49 @@ import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.util.LogUtils;
+import org.apache.log4j.Level;
import org.junit.Assert;
import org.junit.Test;
public class CoMapTest implements Serializable {
private static final long serialVersionUID = 1L;
-
- private static Set<String> result = new HashSet<String>();
+
+ private static Set<String> result;
private static Set<String> expected = new HashSet<String>();
+ private final static class EmptySink extends SinkFunction<Tuple1<Boolean>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void invoke(Tuple1<Boolean> tuple) {
+ }
+ }
+
+ private final static class MyCoMap extends
+ CoMapFunction<Tuple1<String>, Tuple1<Integer>, Tuple1<Boolean>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public Tuple1<Boolean> map1(Tuple1<String> value) {
+ result.add(value.f0);
+ return new Tuple1<Boolean>(true);
+ }
+
+ @Override
+ public Tuple1<Boolean> map2(Tuple1<Integer> value) {
+ result.add(value.f0.toString());
+ return new Tuple1<Boolean>(false);
+ }
+ }
+
@Test
public void test() {
+ LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+
+ result = new HashSet<String>();
+
expected.add("a");
expected.add("b");
expected.add("c");
@@ -46,31 +78,36 @@ public class CoMapTest implements Serializable {
expected.add("2");
expected.add("3");
expected.add("4");
-
+
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
+
DataStream<Tuple1<Integer>> ds1 = env.fromElements(1, 2, 3, 4);
@SuppressWarnings("unused")
- DataStream<Tuple1<Boolean>> ds2 = env.fromElements("a", "b", "c").coMapWith(new CoMapFunction<Tuple1<String>, Tuple1<Integer>, Tuple1<Boolean>>() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Tuple1<Boolean> map1(Tuple1<String> value) {
- System.out.println("1: " + value);
- result.add(value.f0);
- return new Tuple1<Boolean>(true);
- }
-
- @Override
- public Tuple1<Boolean> map2(Tuple1<Integer> value) {
- System.out.println("2: " +value);
- result.add(value.f0.toString());
- return new Tuple1<Boolean>(false);
- }
- }, ds1)
- .print();
+ DataStream<Tuple1<Boolean>> ds2 = env.fromElements("a", "b", "c")
+ .coMapWith(new MyCoMap(), ds1).addSink(new EmptySink());
+
+ env.executeTest(32);
+ Assert.assertArrayEquals(expected.toArray(), result.toArray());
+ }
+
+ @Test
+ public void multipleInputTest() {
+ result = new HashSet<String>();
+
+ LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+ DataStream<Tuple1<Integer>> ds1 = env.fromElements(1, 3);
+ @SuppressWarnings("unchecked")
+ DataStream<Tuple1<Integer>> ds2 = env.fromElements(2, 4).connectWith(ds1);
+
+ DataStream<Tuple1<String>> ds3 = env.fromElements("a", "b");
+ @SuppressWarnings({ "unused", "unchecked" })
+ DataStream<Tuple1<Boolean>> ds4 = env.fromElements("c").connectWith(ds3).coMapWith(new MyCoMap(),
+
+ ds2).addSink(new EmptySink());
+
env.executeTest(32);
Assert.assertArrayEquals(expected.toArray(), result.toArray());
}
[42/51] [abbrv] git commit: [streaming] Implemented sliding window
and batchReduce
Posted by se...@apache.org.
[streaming] Implemented sliding window and batchReduce
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/e089959c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/e089959c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/e089959c
Branch: refs/heads/master
Commit: e089959c0e4bb3e08a95b1b6b9076935c87a5a02
Parents: 309727e
Author: ghermann <re...@gmail.com>
Authored: Thu Aug 7 20:33:50 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:43 2014 +0200
----------------------------------------------------------------------
.../streaming/api/datastream/DataStream.java | 41 +++++-
.../operator/BatchReduceInvokable.java | 135 ++++++++---------
.../operator/WindowReduceInvokable.java | 118 ++++++---------
.../streaming/state/SlidingWindowState.java | 42 ++++--
.../state/SlidingWindowStateIterator.java | 41 +++++-
.../api/invokable/operator/BatchReduceTest.java | 143 +++++++++++++++----
.../streaming/state/SlidingWindowStateTest.java | 77 ++++++++++
7 files changed, 396 insertions(+), 201 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e089959c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index bec55e0..531f43c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -326,7 +326,34 @@ public abstract class DataStream<OUT> {
int batchSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<OUT, R>(reducer,
- batchSize));
+ batchSize, batchSize));
+ }
+
+ /**
+ * Applies a reduce transformation on preset sliding chunks of the
+ * DataStream. The transformation calls a {@link GroupReduceFunction} for
+ * each tuple batch of the predefined size. The tuple batch gets slid by the
+ * given number of tuples. Each GroupReduceFunction call can return any
+ * number of elements including none. The user can also extend
+ * {@link RichGroupReduceFunction} to gain access to other features provided
+ * by the {@link RichFuntion} interface.
+ *
+ *
+ * @param reducer
+ * The GroupReduceFunction that is called for each tuple batch.
+ * @param batchSize
+ * The number of tuples grouped together in the batch.
+ * @param slideSize
+ * The number of tuples the batch is slid by.
+ * @param <R>
+ * output type
+ * @return The transformed {@link DataStream}.
+ */
+ public <R> SingleOutputStreamOperator<R, ?> batchReduce(GroupReduceFunction<OUT, R> reducer,
+ int batchSize, int slideSize) {
+ return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
+ GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<OUT, R>(reducer,
+ batchSize, slideSize));
}
/**
@@ -342,7 +369,8 @@ public abstract class DataStream<OUT> {
* @param reducer
* The GroupReduceFunction that is called for each time window.
* @param windowSize
- * The time window to run the reducer on, in milliseconds.
+ * SingleOutputStreamOperator The time window to run the reducer
+ * on, in milliseconds.
* @param <R>
* output type
* @return The transformed DataStream.
@@ -351,7 +379,14 @@ public abstract class DataStream<OUT> {
long windowSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<OUT, R>(reducer,
- windowSize));
+ windowSize, windowSize, windowSize));
+ }
+
+ public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
+ long windowSize, long slideInterval, long timeUnitInMillis) {
+ return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
+ GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<OUT, R>(reducer,
+ windowSize, slideInterval, timeUnitInMillis));
}
/**
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e089959c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 4e0a7a5..ffd4f1a 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -20,106 +20,87 @@
package org.apache.flink.streaming.api.invokable.operator;
import java.util.ArrayList;
-import java.util.List;
+import org.apache.commons.math.util.MathUtils;
import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.state.SlidingWindowState;
public class BatchReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private int batchSize;
-
- public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, int batchSize) {
- super(reduceFunction);
+ private int slideSize;
+ private int granularity;
+ private boolean emitted;
+ private transient SlidingWindowState<IN> state;
+
+ public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, int batchSize,
+ int slideSize) {
+ super(reduceFunction);
this.reducer = reduceFunction;
this.batchSize = batchSize;
+ this.slideSize = slideSize;
+ this.granularity = MathUtils.gcd(batchSize, slideSize);
}
@Override
protected void immutableInvoke() throws Exception {
- List<IN> tupleBatch = new ArrayList<IN>();
- boolean batchStart;
- int counter = 0;
-
- while (loadNextRecord() != null) {
- batchStart = true;
- do {
- if (batchStart) {
- batchStart = false;
- } else {
- reuse = loadNextRecord();
- if (reuse == null) {
- break;
- }
- }
- counter++;
- tupleBatch.add(reuse.getObject());
- resetReuse();
- } while (counter < batchSize);
- reducer.reduce(tupleBatch, collector);
- tupleBatch.clear();
- counter = 0;
- }
-
- }
-
- @Override
- protected void mutableInvoke() throws Exception {
- userIterator = new CounterIterator();
-
- do {
- if (userIterator.hasNext()) {
- reducer.reduce(userIterable, collector);
- userIterator.reset();
+ reuse = loadNextRecord();
+ ArrayList<IN> list;
+
+ while (!state.isFull()) {
+ list = new ArrayList<IN>(granularity);
+ try {
+ state.pushBack(fillArray(list));
+ } catch (NullPointerException e) {
+ throw new RuntimeException("DataStream length must be greater than batchsize");
}
- } while (reuse != null);
- }
-
- private class CounterIterator implements BatchIterator<IN> {
- private int counter;
- private boolean loadedNext;
-
- public CounterIterator() {
- counter = 1;
}
- @Override
- public boolean hasNext() {
- if (counter > batchSize) {
- return false;
- } else if (!loadedNext) {
- loadNextRecord();
- loadedNext = true;
- }
- return (reuse != null);
- }
+ boolean go = reduce();
- @Override
- public IN next() {
- if (hasNext()) {
- counter++;
- loadedNext = false;
- return reuse.getObject();
+ while (go) {
+ if (state.isEmittable()) {
+ go = reduce();
} else {
- counter++;
- loadedNext = false;
- return null;
+ list = (ArrayList<IN>) state.popFront();
+ list.clear();
+ state.pushBack(fillArray(list));
+ emitted = false;
+ go = reuse != null;
}
}
-
- public void reset() {
- for (int i = 0; i < (batchSize - counter); i++) {
- loadNextRecord();
- }
- loadNextRecord();
- loadedNext = true;
- counter = 1;
+ if (!emitted) {
+ reduce();
}
+ }
- @Override
- public void remove() {
+ private boolean reduce() throws Exception {
+ userIterator = state.getIterator();
+ reducer.reduce(userIterable, collector);
+ emitted = true;
+ return reuse != null;
+ }
- }
+ private ArrayList<IN> fillArray(ArrayList<IN> list) {
+ int counter = 0;
+ do {
+ counter++;
+ list.add(reuse.getObject());
+ resetReuse();
+ } while ((reuse = loadNextRecord()) != null && counter < granularity);
+ return list;
+ }
+ @Override
+ protected void mutableInvoke() throws Exception {
+ throw new RuntimeException("Reducing mutable sliding batch is not supported.");
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception{
+ super.open(parameters);
+ this.state = new SlidingWindowState<IN>(batchSize, slideSize, granularity);
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e089959c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index 3405641..cbc242c 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -20,105 +20,71 @@
package org.apache.flink.streaming.api.invokable.operator;
import java.util.ArrayList;
-import java.util.List;
import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.state.SlidingWindowState;
public class WindowReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private long windowSize;
+ private long slideInterval;
+ private long timeUnitInMillis;
+ private transient SlidingWindowState<IN> state;
volatile boolean isRunning;
- boolean window;
- public WindowReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize) {
+ public WindowReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize,
+ long slideInterval, long timeUnitInMillis) {
super(reduceFunction);
- this.reducer = reduceFunction;
this.windowSize = windowSize;
- this.window = true;
+ this.slideInterval = slideInterval;
+ this.timeUnitInMillis = timeUnitInMillis;
}
protected void immutableInvoke() throws Exception {
- List<IN> tupleBatch = new ArrayList<IN>();
- boolean batchStart;
-
- long startTime = System.currentTimeMillis();
- while (loadNextRecord() != null) {
- batchStart = true;
- do {
- if (batchStart) {
- batchStart = false;
- } else {
- reuse = loadNextRecord();
- if (reuse == null) {
- break;
- }
- }
- tupleBatch.add(reuse.getObject());
- resetReuse();
- } while (System.currentTimeMillis() - startTime < windowSize);
- reducer.reduce(tupleBatch, collector);
- tupleBatch.clear();
- startTime = System.currentTimeMillis();
+ if ((reuse = loadNextRecord()) == null) {
+ throw new RuntimeException("DataStream must not be empty");
}
- }
-
- protected void mutableInvoke() throws Exception {
- userIterator = new WindowIterator();
-
- do {
- if (userIterator.hasNext()) {
- reducer.reduce(userIterable, collector);
- userIterator.reset();
- }
- } while (reuse != null);
- }
-
- private class WindowIterator implements BatchIterator<IN> {
-
- private boolean loadedNext;
- private long startTime;
-
- public WindowIterator() {
- startTime = System.currentTimeMillis();
+ while (reuse != null && !state.isFull()) {
+ collectOneTimeUnit();
}
+ reduce();
- @Override
- public boolean hasNext() {
- if (System.currentTimeMillis() - startTime > windowSize) {
- return false;
- } else if (!loadedNext) {
- loadNextRecord();
- loadedNext = true;
- }
- return (reuse != null);
- }
-
- @Override
- public IN next() {
- if (hasNext()) {
- loadedNext = false;
- return reuse.getObject();
- } else {
- loadedNext = false;
- return reuse.getObject();
+ while (reuse != null) {
+ for (int i = 0; i < slideInterval / timeUnitInMillis; i++) {
+ collectOneTimeUnit();
}
+ reduce();
}
+ }
- public void reset() {
- while (System.currentTimeMillis() - startTime < windowSize) {
- loadNextRecord();
- }
- loadNextRecord();
- loadedNext = true;
- startTime = System.currentTimeMillis();
- }
+ private void collectOneTimeUnit() {
+ ArrayList<IN> list;
+ list = new ArrayList<IN>();
+ long startTime = System.currentTimeMillis();
- @Override
- public void remove() {
+ do {
+ list.add(reuse.getObject());
+ resetReuse();
+ } while ((reuse = loadNextRecord()) != null
+ && System.currentTimeMillis() - startTime < timeUnitInMillis);
+ state.pushBack(list);
+ }
- }
+ private boolean reduce() throws Exception {
+ userIterator = state.forceGetIterator();
+ reducer.reduce(userIterable, collector);
+ return reuse != null;
+ }
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ super.open(parameters);
+ this.state = new SlidingWindowState<IN>(windowSize, slideInterval, timeUnitInMillis);
}
+ protected void mutableInvoke() throws Exception {
+ throw new RuntimeException("Reducing mutable sliding window is not supported.");
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e089959c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
index 600b69f..a062ba9 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
@@ -20,43 +20,43 @@
package org.apache.flink.streaming.state;
import java.io.Serializable;
-import java.util.ArrayList;
+import java.util.List;
import org.apache.commons.collections.buffer.CircularFifoBuffer;
-import org.apache.flink.api.java.tuple.Tuple;
-
/**
* The window state for window operator. To be general enough, this class
* implements a count based window operator. It is possible for the user to
* compose time based window operator by extending this class by splitting the
* stream into multiple mini batches.
*/
-public class SlidingWindowState<InTuple extends Tuple> implements Serializable {
+public class SlidingWindowState<T> implements Serializable {
private static final long serialVersionUID = -2376149970115888901L;
- private int currentRecordCount;
+ private long currentRecordCount;
private int fullRecordCount;
private int slideRecordCount;
+ private SlidingWindowStateIterator<T> iterator;
- CircularFifoBuffer buffer;
+ private CircularFifoBuffer buffer;
- public SlidingWindowState(int windowSize, int slidingStep, int computeGranularity) {
+ public SlidingWindowState(long windowSize, long slideInterval, long timeUnitInMillis) {
this.currentRecordCount = 0;
// here we assume that windowSize and slidingStep is divisible by
- // computeGranularity.
- this.fullRecordCount = windowSize / computeGranularity;
- this.slideRecordCount = slidingStep / computeGranularity;
+ // computationGranularity.
+ this.fullRecordCount = (int) (windowSize / timeUnitInMillis);
+ this.slideRecordCount = (int) (slideInterval / timeUnitInMillis);
this.buffer = new CircularFifoBuffer(fullRecordCount);
+ this.iterator = new SlidingWindowStateIterator<T>(buffer);
}
- public void pushBack(ArrayList<InTuple> tupleArray) {
- buffer.add(tupleArray);
+ public void pushBack(List<T> array) {
+ buffer.add(array);
currentRecordCount += 1;
}
@SuppressWarnings("unchecked")
- public ArrayList<InTuple> popFront() {
- ArrayList<InTuple> frontRecord = (ArrayList<InTuple>) buffer.get();
+ public List<T> popFront() {
+ List<T> frontRecord = (List<T>) buffer.get();
buffer.remove();
return frontRecord;
}
@@ -65,6 +65,20 @@ public class SlidingWindowState<InTuple extends Tuple> implements Serializable {
return currentRecordCount >= fullRecordCount;
}
+ public SlidingWindowStateIterator<T> getIterator() {
+ if (isFull()) {
+ iterator.reset();
+ return iterator;
+ } else {
+ return null;
+ }
+ }
+
+ public SlidingWindowStateIterator<T> forceGetIterator() {
+ iterator.reset();
+ return iterator;
+ }
+
public boolean isEmittable() {
if (currentRecordCount == fullRecordCount + slideRecordCount) {
currentRecordCount -= slideRecordCount;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e089959c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
index 69c9a48..6033276 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
@@ -19,18 +19,45 @@
package org.apache.flink.streaming.state;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import java.util.Collection;
+import java.util.Iterator;
-public class SlidingWindowStateIterator<K>{
+import org.apache.commons.collections.buffer.CircularFifoBuffer;
+import org.apache.flink.streaming.api.invokable.operator.BatchIterator;
+
+public class SlidingWindowStateIterator<T> implements BatchIterator<T> {
+
+ private CircularFifoBuffer buffer;
+ private Iterator<Collection<T>> iterator;
+ private Iterator<T> subIterator;
+
+ public SlidingWindowStateIterator(CircularFifoBuffer buffer) {
+ this.buffer = buffer;
+ }
public boolean hasNext() {
- return false;
+ return subIterator.hasNext();
}
- public Tuple2<K, StreamRecord<Tuple>> next() {
- return null;
+ public T next() {
+ T nextElement = subIterator.next();
+ if (!subIterator.hasNext()) {
+ if (iterator.hasNext()) {
+ subIterator = iterator.next().iterator();
+ }
+ }
+ return nextElement;
}
+ @Override
+ public void remove() {
+ throw new RuntimeException("Cannot use remove on reducing iterator.");
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void reset() {
+ iterator = buffer.iterator();
+ subIterator = iterator.next().iterator();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e089959c/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
index c91878b..bf7621e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -22,10 +22,14 @@ package org.apache.flink.streaming.api.invokable.operator;
import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
@@ -39,64 +43,155 @@ public class BatchReduceTest {
private static ArrayList<Double> avgs = new ArrayList<Double>();
private static final int BATCH_SIZE = 5;
- private static final int PARALlELISM = 1;
+ private static final int PARALLELISM = 1;
private static final long MEMORYSIZE = 32;
- public static final class MyBatchReduce implements
- GroupReduceFunction<Tuple1<Double>, Tuple1<Double>> {
+ public static final class MyBatchReduce implements GroupReduceFunction<Double, Double> {
private static final long serialVersionUID = 1L;
@Override
- public void reduce(Iterable<Tuple1<Double>> values, Collector<Tuple1<Double>> out)
- throws Exception {
+ public void reduce(Iterable<Double> values, Collector<Double> out) throws Exception {
Double sum = 0.;
Double count = 0.;
- for (Tuple1<Double> value : values) {
- sum += value.f0;
+ for (Double value : values) {
+ sum += value;
count++;
}
if (count > 0) {
- out.collect(new Tuple1<Double>(sum / count));
+ out.collect(new Double(sum / count));
}
}
}
- public static final class MySink implements SinkFunction<Tuple1<Double>> {
+ public static final class MySink implements SinkFunction<Double> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Double> tuple) {
- avgs.add(tuple.f0);
+ public void invoke(Double tuple) {
+ avgs.add(tuple);
}
}
- public static final class MySource implements SourceFunction<Tuple1<Double>> {
+ public static final class MySource implements SourceFunction<Double> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Collector<Tuple1<Double>> collector) {
+ public void invoke(Collector<Double> collector) {
for (Double i = 1.; i <= 100; i++) {
- collector.collect(new Tuple1<Double>(i));
+ collector.collect(new Double(i));
}
}
}
- @Test
- public void test() throws Exception {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+ public static final class MySlidingBatchReduce implements RichFunction,
+ GroupReduceFunction<Long, String> {
+ private static final long serialVersionUID = 1L;
- LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(PARALlELISM);
+ double startTime;
- @SuppressWarnings("unused")
- DataStream<Tuple1<Double>> dataStream = env.addSource(new MySource())
- .batchReduce(new MyBatchReduce(), BATCH_SIZE).addSink(new MySink());
+ @Override
+ public void reduce(Iterable<Long> values, Collector<String> out) throws Exception {
+ for (Long value : values) {
+ out.collect(value.toString());
+ }
+ out.collect(END_OF_BATCH);
+ }
- env.executeTest(MEMORYSIZE);
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ startTime = (double) System.currentTimeMillis() / 1000;
+ }
+
+ @Override
+ public void close() throws Exception {
+ }
+
+ @Override
+ public RuntimeContext getRuntimeContext() {
+ return null;
+ }
+
+ @Override
+ public void setRuntimeContext(RuntimeContext t) {
+ // TODO Auto-generated method stub
+
+ }
+ }
+
+ private static List<SortedSet<String>> sink = new ArrayList<SortedSet<String>>();
+ private static final String END_OF_BATCH = "end of batch";
+
+ public static final class MySlidingSink implements SinkFunction<String> {
+ private static final long serialVersionUID = 1L;
+
+ SortedSet<String> currentSet = new TreeSet<String>();
+
+ @Override
+ public void invoke(String string) {
+ if (string.equals(END_OF_BATCH)) {
+ sink.add(currentSet);
+ currentSet = new TreeSet<String>();
+ } else {
+ currentSet.add(string);
+ }
+ }
+ }
+
+ private final static int SLIDING_BATCH_SIZE = 9;
+ private final static int SLIDE_SIZE = 6;
+ private static final int SEQUENCE_SIZE = 30;
+ private LocalStreamEnvironment env;
+
+ private void slidingStream() {
+ env.generateSequence(1, SEQUENCE_SIZE)
+ .batchReduce(new MySlidingBatchReduce(), SLIDING_BATCH_SIZE, SLIDE_SIZE)
+ .addSink(new MySlidingSink());
+ }
+
+ private void slidingTest() {
+ int firstInBatch = 1;
+
+ for (SortedSet<String> set : sink) {
+ int to = Math.min(firstInBatch + SLIDING_BATCH_SIZE - 1, SEQUENCE_SIZE);
+ assertEquals(getExpectedSet(to), set);
+ firstInBatch += SLIDE_SIZE;
+ }
+ }
+
+ private void nonSlidingStream() {
+ env.addSource(new MySource()).batchReduce(new MyBatchReduce(), BATCH_SIZE)
+ .addSink(new MySink());
+ }
+
+ private void nonSlidingTest() {
for (int i = 0; i < avgs.size(); i++) {
assertEquals(3.0 + i * BATCH_SIZE, avgs.get(i), 0);
}
}
+
+ @Test
+ public void test() {
+ LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+
+ env = StreamExecutionEnvironment.createLocalEnvironment(PARALLELISM);
+
+ slidingStream();
+ nonSlidingStream();
+
+ env.executeTest(MEMORYSIZE);
+
+ slidingTest();
+ nonSlidingTest();
+ }
+
+ private SortedSet<String> getExpectedSet(int to) {
+ SortedSet<String> expectedSet = new TreeSet<String>();
+ for (int i = to; i > to - SLIDING_BATCH_SIZE; i--) {
+ expectedSet.add(Integer.toString(i));
+ }
+ return expectedSet;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e089959c/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
new file mode 100644
index 0000000..15902f4
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
@@ -0,0 +1,77 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.state;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.junit.Test;
+
+public class SlidingWindowStateTest {
+
+ private final static int SLIDING_BATCH_SIZE = 3;
+ private final static int SLIDE_SIZE = 2;
+ private static final int UNIT = 1;
+
+ @Test
+ public void test() {
+ SlidingWindowState<Integer> state = new SlidingWindowState<Integer>(SLIDING_BATCH_SIZE,
+ SLIDE_SIZE, UNIT);
+ state.pushBack(Arrays.asList(0));
+ state.pushBack(Arrays.asList(1));
+ assertEquals(false, state.isFull());
+ state.pushBack(Arrays.asList(2));
+ assertTrue(state.isFull());
+
+ SlidingWindowStateIterator<Integer> iterator = state.getIterator();
+
+ SortedSet<Integer> actualSet = new TreeSet<Integer>();
+ while (iterator.hasNext()) {
+ actualSet.add(iterator.next());
+ }
+ assertEquals(getExpectedSet(0, 2), actualSet);
+ actualSet.clear();
+
+ state.pushBack(Arrays.asList(3));
+ assertEquals(false, state.isEmittable());
+ state.pushBack(Arrays.asList(4));
+ assertTrue(state.isEmittable());
+
+ iterator = state.getIterator();
+
+ while (iterator.hasNext()) {
+ actualSet.add(iterator.next());
+ }
+ assertEquals(getExpectedSet(2, 4), actualSet);
+ }
+
+ private SortedSet<Integer> getExpectedSet(int from, int to) {
+ SortedSet<Integer> expectedSet = new TreeSet<Integer>();
+ for (int i = from; i <= to; i++) {
+ expectedSet.add(i);
+ }
+ return expectedSet;
+ }
+
+}
[30/51] [abbrv] git commit: [streaming] DataStream type refactor
Posted by se...@apache.org.
[streaming] DataStream type refactor
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/f932700a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/f932700a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/f932700a
Branch: refs/heads/master
Commit: f932700ae4e7e6110b729afa8530ac029ed0b700
Parents: 910f74d
Author: gyfora <gy...@gmail.com>
Authored: Sun Aug 3 22:47:23 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:50 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 509 +++++++++----------
.../streaming/api/IterativeDataStream.java | 19 +-
.../flink/streaming/api/SplitDataStream.java | 7 +-
.../flink/streaming/api/StreamOperator.java | 8 +-
.../api/collector/DirectedOutputTest.java | 59 ++-
5 files changed, 318 insertions(+), 284 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f932700a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index c648ab2..10f8114 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -77,16 +77,14 @@ public class DataStream<T> {
protected String id;
protected int degreeOfParallelism;
protected String userDefinedName;
- protected List<String> connectIDs;
- protected List<StreamPartitioner<T>> partitioners;
- protected boolean iterationflag;
- protected Integer iterationID;
+ protected StreamPartitioner<T> partitioner;
+ protected List<DataStream<T>> connectedStreams;
protected JobGraphBuilder jobGraphBuilder;
/**
* Create a new {@link DataStream} in the given execution environment with
- * partitioning set to shuffle by default.
+ * partitioning set to forward by default.
*
* @param environment
* StreamExecutionEnvironment
@@ -104,7 +102,9 @@ public class DataStream<T> {
this.environment = environment;
this.degreeOfParallelism = environment.getDegreeOfParallelism();
this.jobGraphBuilder = environment.getJobGraphBuilder();
- initConnections();
+ this.partitioner = new ForwardPartitioner<T>();
+ this.connectedStreams = new ArrayList<DataStream<T>>();
+ this.connectedStreams.add(this.copy());
}
/**
@@ -118,11 +118,13 @@ public class DataStream<T> {
this.id = dataStream.id;
this.degreeOfParallelism = dataStream.degreeOfParallelism;
this.userDefinedName = dataStream.userDefinedName;
- this.connectIDs = new ArrayList<String>(dataStream.connectIDs);
- this.partitioners = new ArrayList<StreamPartitioner<T>>(dataStream.partitioners);
- this.iterationflag = dataStream.iterationflag;
- this.iterationID = dataStream.iterationID;
+ this.partitioner = dataStream.partitioner;
this.jobGraphBuilder = dataStream.jobGraphBuilder;
+ this.connectedStreams = new ArrayList<DataStream<T>>();
+ for (DataStream<T> stream : dataStream.connectedStreams) {
+ this.connectedStreams.add(stream.copy());
+ }
+
}
/**
@@ -142,17 +144,6 @@ public class DataStream<T> {
}
/**
- * Initialize the connection and partitioning among the connected
- * {@link DataStream}s.
- */
- private void initConnections() {
- connectIDs = new ArrayList<String>();
- connectIDs.add(getId());
- partitioners = new ArrayList<StreamPartitioner<T>>();
- partitioners.add(new ForwardPartitioner<T>());
- }
-
- /**
* Returns the ID of the {@link DataStream}.
*
* @return ID of the DataStream
@@ -162,31 +153,12 @@ public class DataStream<T> {
}
/**
- * Sets the mutability of the operator represented by the DataStream. If the
- * operator is set to mutable, the tuples received in the user defined
- * functions, will be reused after the function call. Setting an operator to
- * mutable greatly reduces garbage collection overhead and thus scalability.
- *
- * @param isMutable
- * The mutability of the operator.
- * @return The DataStream with mutability set.
- */
- public DataStream<T> setMutability(boolean isMutable) {
- jobGraphBuilder.setMutability(id, isMutable);
- return this;
- }
-
- /**
- * Sets the maximum time frequency (ms) for the flushing of the output
- * buffer. By default the output buffers flush only when they are full.
+ * Gets the degree of parallelism for this operator.
*
- * @param timeoutMillis
- * The maximum time between two output flushes.
- * @return The DataStream with buffer timeout set.
+ * @return The parallelism set for this operator.
*/
- public DataStream<T> setBufferTimeout(long timeoutMillis) {
- jobGraphBuilder.setBufferTimeout(id, timeoutMillis);
- return this;
+ public int getParallelism() {
+ return this.degreeOfParallelism;
}
/**
@@ -202,39 +174,37 @@ public class DataStream<T> {
throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
}
this.degreeOfParallelism = dop;
-
+
jobGraphBuilder.setParallelism(id, degreeOfParallelism);
-
+
return this;
}
/**
- * Gets the degree of parallelism for this operator.
+ * Sets the mutability of the operator represented by the DataStream. If the
+ * operator is set to mutable, the tuples received in the user defined
+ * functions, will be reused after the function call. Setting an operator to
+ * mutable greatly reduces garbage collection overhead and thus scalability.
*
- * @return The parallelism set for this operator.
+ * @param isMutable
+ * The mutability of the operator.
+ * @return The DataStream with mutability set.
*/
- public int getParallelism() {
- return this.degreeOfParallelism;
+ public DataStream<T> setMutability(boolean isMutable) {
+ jobGraphBuilder.setMutability(id, isMutable);
+ return this;
}
/**
- * Gives the data transformation(vertex) a user defined name in order to use
- * with directed outputs. The {@link OutputSelector} of the input vertex
- * should use this name for directed emits.
+ * Sets the maximum time frequency (ms) for the flushing of the output
+ * buffer. By default the output buffers flush only when they are full.
*
- * @param name
- * The name to set
- * @return The named DataStream.
+ * @param timeoutMillis
+ * The maximum time between two output flushes.
+ * @return The DataStream with buffer timeout set.
*/
- protected DataStream<T> name(String name) {
- // TODO copy DataStream?
- if (name == "") {
- throw new IllegalArgumentException("User defined name must not be empty string");
- }
-
- userDefinedName = name;
- jobGraphBuilder.setUserDefinedName(id, name);
-
+ public DataStream<T> setBufferTimeout(long timeoutMillis) {
+ jobGraphBuilder.setBufferTimeout(id, timeoutMillis);
return this;
}
@@ -258,19 +228,6 @@ public class DataStream<T> {
}
/**
- * Connects two DataStreams
- *
- * @param returnStream
- * The other DataStream will connected to this
- * @param stream
- * This DataStream will be connected to returnStream
- */
- private void addConnection(DataStream<T> returnStream, DataStream<T> stream) {
- returnStream.connectIDs.addAll(stream.connectIDs);
- returnStream.partitioners.addAll(stream.partitioners);
- }
-
- /**
* Operator used for directing tuples to specific named outputs. Sets an
* {@link OutputSelector} for the vertex. The tuples emitted from this
* vertex will be sent to the output names selected by the OutputSelector.
@@ -282,8 +239,9 @@ public class DataStream<T> {
*/
public SplitDataStream<T> split(OutputSelector<T> outputSelector) {
try {
- for (String id : connectIDs) {
- jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
+ for (DataStream<T> stream : connectedStreams) {
+ jobGraphBuilder.setOutputSelector(stream.id,
+ SerializationUtils.serialize(outputSelector));
}
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize OutputSelector");
@@ -349,16 +307,6 @@ public class DataStream<T> {
return setConnectionType(new DistributePartitioner<T>());
}
- private DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
- DataStream<T> returnStream = this.copy();
-
- for (int i = 0; i < returnStream.partitioners.size(); i++) {
- returnStream.partitioners.set(i, partitioner);
- }
-
- return returnStream;
- }
-
/**
* Applies a Map transformation on a {@link DataStream}. The transformation
* calls a {@link RichMapFunction} for each element of the DataStream. Each
@@ -377,28 +325,6 @@ public class DataStream<T> {
}
/**
- * Applies a CoMap transformation on two separate {@link DataStream}s. The
- * transformation calls a {@link CoMapFunction#map1(Tuple)} for each element
- * of the first DataStream (on which .coMapWith was called) and
- * {@link CoMapFunction#map2(Tuple)} for each element of the second
- * DataStream. Each CoMapFunction call returns exactly one element.
- *
- * @param coMapper
- * The CoMapFunction used to jointly transform the two input
- * DataStreams
- * @param otherStream
- * The DataStream that will be transformed with
- * {@link CoMapFunction#map2(Tuple)}
- * @return The transformed DataStream
- */
- public <T2, R> DataStream<R> coMapWith(CoMapFunction<T, T2, R> coMapper,
- DataStream<T2> otherStream) {
- return addCoFunction("coMap", this.copy(), otherStream.copy(), coMapper,
- new FunctionTypeWrapper<T, T2, R>(coMapper, CoMapFunction.class, 0, 1, 2),
- new CoMapInvokable<T, T2, R>(coMapper));
- }
-
- /**
* Applies a FlatMap transformation on a {@link DataStream}. The
* transformation calls a {@link RichFlatMapFunction} for each element of
* the DataStream. Each RichFlatMapFunction call can return any number of
@@ -418,19 +344,25 @@ public class DataStream<T> {
}
/**
- * Applies a Filter transformation on a {@link DataStream}. The
- * transformation calls a {@link RichFilterFunction} for each element of the
- * DataStream and retains only those element for which the function returns
- * true. Elements for which the function returns false are filtered.
+ * Applies a CoMap transformation on two separate {@link DataStream}s. The
+ * transformation calls a {@link CoMapFunction#map1(Tuple)} for each element
+ * of the first DataStream (on which .coMapWith was called) and
+ * {@link CoMapFunction#map2(Tuple)} for each element of the second
+ * DataStream. Each CoMapFunction call returns exactly one element.
*
- * @param filter
- * The RichFilterFunction that is called for each element of the
- * DataSet.
- * @return The filtered DataStream.
+ * @param coMapper
+ * The CoMapFunction used to jointly transform the two input
+ * DataStreams
+ * @param otherStream
+ * The DataStream that will be transformed with
+ * {@link CoMapFunction#map2(Tuple)}
+ * @return The transformed DataStream
*/
- public StreamOperator<T, T> filter(RichFilterFunction<T> filter) {
- return addFunction("filter", filter, new FunctionTypeWrapper<T, Tuple, T>(filter,
- RichFilterFunction.class, 0, -1, 0), new FilterInvokable<T>(filter));
+ public <T2, R> DataStream<R> coMapWith(CoMapFunction<T, T2, R> coMapper,
+ DataStream<T2> otherStream) {
+ return addCoFunction("coMap", this.copy(), otherStream.copy(), coMapper,
+ new FunctionTypeWrapper<T, T2, R>(coMapper, CoMapFunction.class, 0, 1, 2),
+ new CoMapInvokable<T, T2, R>(coMapper));
}
/**
@@ -480,115 +412,19 @@ public class DataStream<T> {
}
/**
- * Internal function for passing the user defined functions to the JobGraph
- * of the job.
- *
- * @param functionName
- * name of the function
- * @param function
- * the user defined function
- * @param functionInvokable
- * the wrapping JobVertex instance
- * @param <T>
- * type of the input stream
- * @param <R>
- * type of the return stream
- * @return the data stream constructed
- */
- private <R> StreamOperator<T, R> addFunction(String functionName,
- final AbstractRichFunction function, TypeSerializerWrapper<T, Tuple, R> typeWrapper,
- UserTaskInvokable<T, R> functionInvokable) {
-
- DataStream<T> inputStream = this.copy();
- StreamOperator<T, R> returnStream = new StreamOperator<T, R>(environment, functionName);
-
- try {
- jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, typeWrapper,
- functionName, SerializationUtils.serialize(function), degreeOfParallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize user defined function");
- }
-
- connectGraph(inputStream, returnStream.getId(), 0);
-
- if (inputStream.iterationflag) {
- returnStream.addIterationSource(inputStream.iterationID.toString());
- inputStream.iterationflag = false;
- }
-
- if (inputStream instanceof NamedDataStream) {
- returnStream.name(inputStream.userDefinedName);
- }
-
- return returnStream;
- }
-
- protected <T1, T2, R> DataStream<R> addCoFunction(String functionName,
- DataStream<T1> inputStream1, DataStream<T2> inputStream2,
- final AbstractRichFunction function, TypeSerializerWrapper<T1, T2, R> typeWrapper,
- CoInvokable<T1, T2, R> functionInvokable) {
-
- DataStream<R> returnStream = new DataStream<R>(environment, functionName);
-
- try {
- jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, typeWrapper,
- functionName, SerializationUtils.serialize(function), degreeOfParallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize user defined function");
- }
-
- connectGraph(inputStream1, returnStream.getId(), 1);
- connectGraph(inputStream2, returnStream.getId(), 2);
-
- if ((inputStream1 instanceof NamedDataStream) && (inputStream2 instanceof NamedDataStream)) {
- throw new RuntimeException("An operator cannot have two names");
- } else {
- if (inputStream1 instanceof NamedDataStream) {
- returnStream.name(inputStream1.userDefinedName);
- }
-
- if (inputStream2 instanceof NamedDataStream) {
- returnStream.name(inputStream2.userDefinedName);
- }
- }
- // TODO consider iteration
-
- return returnStream;
- }
-
- /**
- * Internal function for assembling the underlying
- * {@link org.apache.flink.nephele.jobgraph.JobGraph} of the job. Connects
- * the outputs of the given input stream to the specified output stream
- * given by the outputID.
- *
- * @param inputStream
- * input data stream
- * @param outputID
- * ID of the output
- * @param typeNumber
- * Number of the type (used at co-functions)
- */
- private <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
- for (int i = 0; i < inputStream.connectIDs.size(); i++) {
- String inputID = inputStream.connectIDs.get(i);
- StreamPartitioner<X> partitioner = inputStream.partitioners.get(i);
-
- jobGraphBuilder.setEdge(inputID, outputID, partitioner, typeNumber);
- }
- }
-
- /**
- * Adds the given sink to this DataStream. Only streams with sinks added
- * will be executed once the {@link StreamExecutionEnvironment#execute()}
- * method is called.
+ * Applies a Filter transformation on a {@link DataStream}. The
+ * transformation calls a {@link RichFilterFunction} for each element of the
+ * DataStream and retains only those element for which the function returns
+ * true. Elements for which the function returns false are filtered.
*
- * @param sinkFunction
- * The object containing the sink's invoke function.
- * @return The closed DataStream.
+ * @param filter
+ * The RichFilterFunction that is called for each element of the
+ * DataSet.
+ * @return The filtered DataStream.
*/
- public DataStream<T> addSink(SinkFunction<T> sinkFunction) {
- return addSink(this.copy(), sinkFunction);
+ public StreamOperator<T, T> filter(RichFilterFunction<T> filter) {
+ return addFunction("filter", filter, new FunctionTypeWrapper<T, Tuple, T>(filter,
+ RichFilterFunction.class, 0, -1, 0), new FilterInvokable<T>(filter));
}
/**
@@ -608,32 +444,6 @@ public class DataStream<T> {
return returnStream;
}
- private DataStream<T> addSink(DataStream<T> inputStream, SinkFunction<T> sinkFunction) {
- return addSink(inputStream, sinkFunction, new FunctionTypeWrapper<T, Tuple, T>(
- sinkFunction, SinkFunction.class, 0, -1, 0));
- }
-
- private DataStream<T> addSink(DataStream<T> inputStream, SinkFunction<T> sinkFunction,
- TypeSerializerWrapper<T, Tuple, T> typeWrapper) {
- DataStream<T> returnStream = new DataStream<T>(environment, "sink");
-
- try {
- jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<T>(sinkFunction),
- typeWrapper, "sink", SerializationUtils.serialize(sinkFunction),
- degreeOfParallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize SinkFunction");
- }
-
- inputStream.connectGraph(inputStream, returnStream.getId(), 0);
-
- if (this.copy() instanceof NamedDataStream) {
- returnStream.name(inputStream.userDefinedName);
- }
-
- return returnStream;
- }
-
/**
* Writes a DataStream to the file specified by path in text format. For
* every element of the DataStream the result of {@link Object#toString()}
@@ -956,10 +766,193 @@ public class DataStream<T> {
protected <R> DataStream<T> addIterationSource(String iterationID) {
DataStream<R> returnStream = new DataStream<R>(environment, "iterationSource");
-
+
jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
degreeOfParallelism);
-
+
return this.copy();
}
+
+ /**
+ * Internal function for passing the user defined functions to the JobGraph
+ * of the job.
+ *
+ * @param functionName
+ * name of the function
+ * @param function
+ * the user defined function
+ * @param functionInvokable
+ * the wrapping JobVertex instance
+ * @param <T>
+ * type of the input stream
+ * @param <R>
+ * type of the return stream
+ * @return the data stream constructed
+ */
+ private <R> StreamOperator<T, R> addFunction(String functionName,
+ final AbstractRichFunction function, TypeSerializerWrapper<T, Tuple, R> typeWrapper,
+ UserTaskInvokable<T, R> functionInvokable) {
+
+ DataStream<T> inputStream = this.copy();
+ StreamOperator<T, R> returnStream = new StreamOperator<T, R>(environment, functionName);
+
+ try {
+ jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, typeWrapper,
+ functionName, SerializationUtils.serialize(function), degreeOfParallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize user defined function");
+ }
+
+ connectGraph(inputStream, returnStream.getId(), 0);
+
+ if (inputStream instanceof IterativeDataStream) {
+ returnStream.addIterationSource(((IterativeDataStream<T>) inputStream).iterationID
+ .toString());
+ }
+
+ if (inputStream instanceof NamedDataStream) {
+ returnStream.name(inputStream.userDefinedName);
+ }
+
+ return returnStream;
+ }
+
+ protected <T1, T2, R> DataStream<R> addCoFunction(String functionName,
+ DataStream<T1> inputStream1, DataStream<T2> inputStream2,
+ final AbstractRichFunction function, TypeSerializerWrapper<T1, T2, R> typeWrapper,
+ CoInvokable<T1, T2, R> functionInvokable) {
+
+ DataStream<R> returnStream = new DataStream<R>(environment, functionName);
+
+ try {
+ jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, typeWrapper,
+ functionName, SerializationUtils.serialize(function), degreeOfParallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize user defined function");
+ }
+
+ connectGraph(inputStream1, returnStream.getId(), 1);
+ connectGraph(inputStream2, returnStream.getId(), 2);
+
+ if ((inputStream1 instanceof NamedDataStream) && (inputStream2 instanceof NamedDataStream)) {
+ throw new RuntimeException("An operator cannot have two names");
+ } else {
+ if (inputStream1 instanceof NamedDataStream) {
+ returnStream.name(inputStream1.userDefinedName);
+ }
+
+ if (inputStream2 instanceof NamedDataStream) {
+ returnStream.name(inputStream2.userDefinedName);
+ }
+ }
+ // TODO consider iteration
+
+ return returnStream;
+ }
+
+ /**
+ * Gives the data transformation(vertex) a user defined name in order to use
+ * with directed outputs. The {@link OutputSelector} of the input vertex
+ * should use this name for directed emits.
+ *
+ * @param name
+ * The name to set
+ * @return The named DataStream.
+ */
+ protected DataStream<T> name(String name) {
+ // TODO copy DataStream?
+ if (name == "") {
+ throw new IllegalArgumentException("User defined name must not be empty string");
+ }
+
+ userDefinedName = name;
+ jobGraphBuilder.setUserDefinedName(id, name);
+
+ return this;
+ }
+
+ /**
+ * Connects two DataStreams
+ *
+ * @param returnStream
+ * The other DataStream will connected to this
+ * @param stream
+ * This DataStream will be connected to returnStream
+ */
+ private void addConnection(DataStream<T> returnStream, DataStream<T> stream) {
+ if ((stream instanceof NamedDataStream) || (returnStream instanceof NamedDataStream)) {
+ if (!returnStream.userDefinedName.equals(stream.userDefinedName)) {
+ throw new RuntimeException("Error: Connected NamedDataStreams must have same names");
+ }
+ }
+ returnStream.connectedStreams.add(stream.copy());
+ }
+
+ private DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
+ DataStream<T> returnStream = this.copy();
+
+ for (DataStream<T> stream : returnStream.connectedStreams) {
+ stream.partitioner = partitioner;
+ }
+
+ return returnStream;
+ }
+
+ /**
+ * Internal function for assembling the underlying
+ * {@link org.apache.flink.nephele.jobgraph.JobGraph} of the job. Connects
+ * the outputs of the given input stream to the specified output stream
+ * given by the outputID.
+ *
+ * @param inputStream
+ * input data stream
+ * @param outputID
+ * ID of the output
+ * @param typeNumber
+ * Number of the type (used at co-functions)
+ */
+ private <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
+ for (DataStream<X> stream : inputStream.connectedStreams) {
+ jobGraphBuilder.setEdge(stream.getId(), outputID, stream.partitioner, typeNumber);
+ }
+ }
+
+ /**
+ * Adds the given sink to this DataStream. Only streams with sinks added
+ * will be executed once the {@link StreamExecutionEnvironment#execute()}
+ * method is called.
+ *
+ * @param sinkFunction
+ * The object containing the sink's invoke function.
+ * @return The closed DataStream.
+ */
+ public DataStream<T> addSink(SinkFunction<T> sinkFunction) {
+ return addSink(this.copy(), sinkFunction);
+ }
+
+ private DataStream<T> addSink(DataStream<T> inputStream, SinkFunction<T> sinkFunction) {
+ return addSink(inputStream, sinkFunction, new FunctionTypeWrapper<T, Tuple, T>(
+ sinkFunction, SinkFunction.class, 0, -1, 0));
+ }
+
+ private DataStream<T> addSink(DataStream<T> inputStream, SinkFunction<T> sinkFunction,
+ TypeSerializerWrapper<T, Tuple, T> typeWrapper) {
+ DataStream<T> returnStream = new DataStream<T>(environment, "sink");
+
+ try {
+ jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<T>(sinkFunction),
+ typeWrapper, "sink", SerializationUtils.serialize(sinkFunction),
+ degreeOfParallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize SinkFunction");
+ }
+
+ inputStream.connectGraph(inputStream, returnStream.getId(), 0);
+
+ if (this.copy() instanceof NamedDataStream) {
+ returnStream.name(inputStream.userDefinedName);
+ }
+
+ return returnStream;
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f932700a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
index bfce834..d5a5f78 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
@@ -31,12 +31,17 @@ import org.apache.flink.streaming.partitioner.ForwardPartitioner;
public class IterativeDataStream<T> extends StreamOperator<T, T> {
static Integer iterationCount = 0;
+ protected Integer iterationID;
protected IterativeDataStream(DataStream<T> dataStream) {
super(dataStream);
iterationID = iterationCount;
iterationCount++;
- iterationflag = true;
+ }
+
+ protected IterativeDataStream(DataStream<T> dataStream, Integer iterationID) {
+ super(dataStream);
+ this.iterationID = iterationID;
}
/**
@@ -68,8 +73,7 @@ public class IterativeDataStream<T> extends StreamOperator<T, T> {
* when used with directed emits
*
*/
- public <R> DataStream<T> closeWith(DataStream<T> iterationResult,
- String iterationName) {
+ public <R> DataStream<T> closeWith(DataStream<T> iterationResult, String iterationName) {
DataStream<R> returnStream = new DataStream<R>(environment, "iterationSink");
jobGraphBuilder.addIterationSink(returnStream.getId(), iterationResult.getId(),
@@ -78,11 +82,16 @@ public class IterativeDataStream<T> extends StreamOperator<T, T> {
jobGraphBuilder.setIterationSourceParallelism(iterationID.toString(),
iterationResult.getParallelism());
- for (int i = 0; i < iterationResult.connectIDs.size(); i++) {
- String inputID = iterationResult.connectIDs.get(i);
+ for (DataStream<T> stream : iterationResult.connectedStreams) {
+ String inputID = stream.getId();
jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<T>(), 0);
}
return iterationResult;
}
+
+ @Override
+ protected DataStream<T> copy() {
+ return new IterativeDataStream<T>(this, iterationID);
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f932700a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
index b4bbe52..627aa42 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SplitDataStream.java
@@ -34,10 +34,9 @@ public class SplitDataStream<T> extends DataStream<T> {
* @return Returns the modified DataStream
*/
public NamedDataStream<T> select(String outputName) {
-
- userDefinedName = outputName;
-
- return new NamedDataStream<T>(this);
+ NamedDataStream<T> returnStream = new NamedDataStream<T>(this);
+ returnStream.userDefinedName = outputName;
+ return returnStream;
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f932700a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
index 7edde1c..5ecd930 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
@@ -19,8 +19,7 @@
package org.apache.flink.streaming.api;
-
-public class StreamOperator<IN, OUT > extends DataStream<OUT> {
+public class StreamOperator<IN, OUT> extends DataStream<OUT> {
protected StreamOperator(StreamExecutionEnvironment environment, String operatorType) {
super(environment, operatorType);
@@ -30,4 +29,9 @@ public class StreamOperator<IN, OUT > extends DataStream<OUT> {
super(dataStream);
}
+ @Override
+ protected DataStream<OUT> copy() {
+ return new StreamOperator<IN, OUT>(this);
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f932700a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index 074992b..e2991b4 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -20,6 +20,7 @@
package org.apache.flink.streaming.api.collector;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
import java.util.Arrays;
import java.util.Collection;
@@ -38,9 +39,9 @@ public class DirectedOutputTest {
static HashSet<Long> evenSet = new HashSet<Long>();
static HashSet<Long> oddSet = new HashSet<Long>();
-
+
private static class PlusTwo extends RichMapFunction<Long, Long> {
-
+
private static final long serialVersionUID = 1L;
@Override
@@ -59,7 +60,7 @@ public class DirectedOutputTest {
evenSet.add(tuple);
}
}
-
+
private static class OddSink extends SinkFunction<Long> {
private static final long serialVersionUID = 1L;
@@ -69,26 +70,24 @@ public class DirectedOutputTest {
oddSet.add(tuple);
}
}
-
-
+
private static class MySelector extends OutputSelector<Long> {
-
+
private static final long serialVersionUID = 1L;
@Override
public void select(Long tuple, Collection<String> outputs) {
int mod = (int) (tuple % 2);
switch (mod) {
- case 0:
- outputs.add("ds1");
- break;
- case 1:
- outputs.add("ds2");
- break;
+ case 0:
+ outputs.add("ds1");
+ break;
+ case 1:
+ outputs.add("ds2");
+ break;
}
}
}
-
@SuppressWarnings("unused")
@Test
@@ -102,11 +101,41 @@ public class DirectedOutputTest {
DataStream<Long> ds3 = s.map(new PlusTwo()).addSink(new OddSink());
env.execute();
-
+
HashSet<Long> expectedEven = new HashSet<Long>(Arrays.asList(4L, 6L, 8L));
HashSet<Long> expectedOdd = new HashSet<Long>(Arrays.asList(3L, 5L, 7L));
-
+
assertEquals(expectedEven, evenSet);
assertEquals(expectedOdd, oddSet);
}
+
+ @SuppressWarnings({ "unchecked" })
+ @Test
+ public void directNamingTest() {
+ LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+ SplitDataStream<Long> s = env.generateSequence(1, 10).split(new MySelector());
+ try {
+ s.select("ds2").connectWith(s.select("ds1"));
+ fail();
+ } catch (Exception e) {
+ // Exception thrown
+ }
+ try {
+ s.shuffle().connectWith(s.select("ds1"));
+ fail();
+ } catch (Exception e) {
+ // Exception thrown
+ }
+ try {
+ s.select("ds2").connectWith(s);
+ fail();
+ } catch (Exception e) {
+ // Exception thrown
+ }
+ s.connectWith(s);
+ s.select("ds2").connectWith(s.select("ds2"));
+
+ }
}
[44/51] [abbrv] git commit: [streaming] DataStream fix to return
DataStreamSink on closing operators
Posted by se...@apache.org.
[streaming] DataStream fix to return DataStreamSink on closing operators
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/74d3742b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/74d3742b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/74d3742b
Branch: refs/heads/master
Commit: 74d3742b6e39c174111d3b0a6177c25b93f9c926
Parents: 1b31f4d
Author: gyfora <gy...@gmail.com>
Authored: Wed Aug 6 17:22:06 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:43 2014 +0200
----------------------------------------------------------------------
.../streaming/api/datastream/DataStream.java | 48 ++++++++++----------
1 file changed, 24 insertions(+), 24 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/74d3742b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 7cab2df..86b3322 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -353,10 +353,10 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream.
*/
- public DataStream<OUT> print() {
+ public DataStreamSink<OUT> print() {
DataStream<OUT> inputStream = this.copy();
PrintSinkFunction<OUT> printFunction = new PrintSinkFunction<OUT>();
- DataStream<OUT> returnStream = addSink(inputStream, printFunction, null);
+ DataStreamSink<OUT> returnStream = addSink(inputStream, printFunction, null);
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
@@ -373,7 +373,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsText(String path) {
+ public DataStreamSink<OUT> writeAsText(String path) {
return writeAsText(this, path, new WriteFormatAsText<OUT>(), 1, null);
}
@@ -390,7 +390,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsText(String path, long millis) {
+ public DataStreamSink<OUT> writeAsText(String path, long millis) {
return writeAsText(this, path, new WriteFormatAsText<OUT>(), millis, null);
}
@@ -408,7 +408,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsText(String path, int batchSize) {
+ public DataStreamSink<OUT> writeAsText(String path, int batchSize) {
return writeAsText(this, path, new WriteFormatAsText<OUT>(), batchSize, null);
}
@@ -430,7 +430,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsText(String path, long millis, OUT endTuple) {
+ public DataStreamSink<OUT> writeAsText(String path, long millis, OUT endTuple) {
return writeAsText(this, path, new WriteFormatAsText<OUT>(), millis, endTuple);
}
@@ -453,7 +453,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsText(String path, int batchSize, OUT endTuple) {
+ public DataStreamSink<OUT> writeAsText(String path, int batchSize, OUT endTuple) {
return writeAsText(this, path, new WriteFormatAsText<OUT>(), batchSize, endTuple);
}
@@ -475,9 +475,9 @@ public abstract class DataStream<OUT> {
*
* @return the data stream constructed
*/
- private DataStream<OUT> writeAsText(DataStream<OUT> inputStream, String path,
+ private DataStreamSink<OUT> writeAsText(DataStream<OUT> inputStream, String path,
WriteFormatAsText<OUT> format, long millis, OUT endTuple) {
- DataStream<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<OUT>(
+ DataStreamSink<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<OUT>(
path, format, millis, endTuple), null);
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
jobGraphBuilder.setMutability(returnStream.getId(), false);
@@ -503,9 +503,9 @@ public abstract class DataStream<OUT> {
*
* @return the data stream constructed
*/
- private DataStream<OUT> writeAsText(DataStream<OUT> inputStream, String path,
+ private DataStreamSink<OUT> writeAsText(DataStream<OUT> inputStream, String path,
WriteFormatAsText<OUT> format, int batchSize, OUT endTuple) {
- DataStream<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<OUT>(
+ DataStreamSink<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<OUT>(
path, format, batchSize, endTuple), null);
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
jobGraphBuilder.setMutability(returnStream.getId(), false);
@@ -522,7 +522,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsCsv(String path) {
+ public DataStreamSink<OUT> writeAsCsv(String path) {
return writeAsCsv(this, path, new WriteFormatAsCsv<OUT>(), 1, null);
}
@@ -539,7 +539,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsCsv(String path, long millis) {
+ public DataStreamSink<OUT> writeAsCsv(String path, long millis) {
return writeAsCsv(this, path, new WriteFormatAsCsv<OUT>(), millis, null);
}
@@ -557,7 +557,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsCsv(String path, int batchSize) {
+ public DataStreamSink<OUT> writeAsCsv(String path, int batchSize) {
return writeAsCsv(this, path, new WriteFormatAsCsv<OUT>(), batchSize, null);
}
@@ -579,7 +579,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsCsv(String path, long millis, OUT endTuple) {
+ public DataStreamSink<OUT> writeAsCsv(String path, long millis, OUT endTuple) {
return writeAsCsv(this, path, new WriteFormatAsCsv<OUT>(), millis, endTuple);
}
@@ -602,7 +602,7 @@ public abstract class DataStream<OUT> {
*
* @return The closed DataStream
*/
- public DataStream<OUT> writeAsCsv(String path, int batchSize, OUT endTuple) {
+ public DataStreamSink<OUT> writeAsCsv(String path, int batchSize, OUT endTuple) {
if (this instanceof SingleOutputStreamOperator) {
((SingleOutputStreamOperator<?, ?>) this).setMutability(false);
}
@@ -627,9 +627,9 @@ public abstract class DataStream<OUT> {
*
* @return the data stream constructed
*/
- private DataStream<OUT> writeAsCsv(DataStream<OUT> inputStream, String path,
+ private DataStreamSink<OUT> writeAsCsv(DataStream<OUT> inputStream, String path,
WriteFormatAsCsv<OUT> format, long millis, OUT endTuple) {
- DataStream<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<OUT>(
+ DataStreamSink<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<OUT>(
path, format, millis, endTuple));
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
jobGraphBuilder.setMutability(returnStream.getId(), false);
@@ -655,9 +655,9 @@ public abstract class DataStream<OUT> {
*
* @return the data stream constructed
*/
- private DataStream<OUT> writeAsCsv(DataStream<OUT> inputStream, String path,
+ private DataStreamSink<OUT> writeAsCsv(DataStream<OUT> inputStream, String path,
WriteFormatAsCsv<OUT> format, int batchSize, OUT endTuple) {
- DataStream<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<OUT>(
+ DataStreamSink<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<OUT>(
path, format, batchSize, endTuple), null);
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
jobGraphBuilder.setMutability(returnStream.getId(), false);
@@ -792,18 +792,18 @@ public abstract class DataStream<OUT> {
* The object containing the sink's invoke function.
* @return The closed DataStream.
*/
- public DataStream<OUT> addSink(SinkFunction<OUT> sinkFunction) {
+ public DataStreamSink<OUT> addSink(SinkFunction<OUT> sinkFunction) {
return addSink(this.copy(), sinkFunction);
}
- private DataStream<OUT> addSink(DataStream<OUT> inputStream, SinkFunction<OUT> sinkFunction) {
+ private DataStreamSink<OUT> addSink(DataStream<OUT> inputStream, SinkFunction<OUT> sinkFunction) {
return addSink(inputStream, sinkFunction, new FunctionTypeWrapper<OUT, Tuple, OUT>(
sinkFunction, SinkFunction.class, 0, -1, 0));
}
- private DataStream<OUT> addSink(DataStream<OUT> inputStream, SinkFunction<OUT> sinkFunction,
+ private DataStreamSink<OUT> addSink(DataStream<OUT> inputStream, SinkFunction<OUT> sinkFunction,
TypeSerializerWrapper<OUT, Tuple, OUT> typeWrapper) {
- DataStream<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink");
+ DataStreamSink<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink");
try {
jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<OUT>(sinkFunction),
[17/51] [abbrv] git commit: [streaming] Moved task configurations to
StreamConfig
Posted by se...@apache.org.
[streaming] Moved task configurations to StreamConfig
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/330d8fd5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/330d8fd5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/330d8fd5
Branch: refs/heads/master
Commit: 330d8fd524ba6cbc1ec96132ee75698d1f0d4af3
Parents: a2c4137
Author: ghermann <re...@gmail.com>
Authored: Thu Jul 24 14:57:28 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:21:11 2014 +0200
----------------------------------------------------------------------
.../flink/streaming/api/StreamConfig.java | 242 +++++++++++++++++++
.../AbstractStreamComponent.java | 53 ++--
.../api/streamcomponent/CoStreamTask.java | 30 +--
.../SingleInputAbstractStreamComponent.java | 13 +-
.../streamcomponent/StreamIterationSink.java | 2 +-
.../streamcomponent/StreamIterationSource.java | 12 +-
.../api/streamcomponent/StreamSink.java | 4 +-
.../api/streamcomponent/StreamSource.java | 14 +-
.../api/streamcomponent/StreamTask.java | 13 +-
.../api/streamcomponent/StreamWindowTask.java | 96 --------
.../examples/window/sum/WindowSumAggregate.java | 67 -----
.../examples/window/sum/WindowSumLocal.java | 43 ----
.../examples/window/sum/WindowSumMultiple.java | 36 ---
.../examples/window/sum/WindowSumSink.java | 31 ---
.../examples/window/sum/WindowSumSource.java | 41 ----
.../wordcount/WindowWordCountCounter.java | 82 -------
.../window/wordcount/WindowWordCountLocal.java | 50 ----
.../window/wordcount/WindowWordCountSink.java | 32 ---
.../wordcount/WindowWordCountSplitter.java | 46 ----
19 files changed, 293 insertions(+), 614 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
new file mode 100644
index 0000000..d677046
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -0,0 +1,242 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.SerializationException;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.collector.OutputSelector;
+import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
+import org.apache.flink.streaming.api.streamcomponent.StreamComponentException;
+import org.apache.flink.streaming.partitioner.ShufflePartitioner;
+import org.apache.flink.streaming.partitioner.StreamPartitioner;
+
+public class StreamConfig {
+ private static final String INPUT_TYPE = "inputType_";
+ private static final String NUMBER_OF_OUTPUTS = "numberOfOutputs";
+ private static final String NUMBER_OF_INPUTS = "numberOfInputs";
+ private static final String OUTPUT_NAME = "outputName_";
+ private static final String PARTITIONER_OBJECT = "partitionerObject_";
+ private static final String USER_DEFINED_NAME = "userDefinedName";
+ private static final String NUMBER_OF_OUTPUT_CHANNELS = "numOfOutputs_";
+ private static final String ITERATION_ID = "iteration-id";
+ private static final String OUTPUT_SELECTOR = "outputSelector";
+ private static final String DIRECTED_EMIT = "directedEmit";
+ private static final String FUNCTION_NAME = "operatorName";
+ private static final String FUNCTION = "operator";
+ private static final String COMPONENT_NAME = "componentName";
+ private static final String SERIALIZEDUDF = "serializedudf";
+ private static final String USER_FUNCTION = "userfunction";
+ private static final String BUFFER_TIMEOUT = "bufferTimeout";
+
+ // DEFAULT VALUES
+
+ private static final boolean DEFAULT_IS_MUTABLE = false;
+
+ private static final long DEFAULT_TIMEOUT = 0;
+
+ // STRINGS
+
+ private static final String MUTABILITY = "isMutable";
+
+ private Configuration config;
+
+ public StreamConfig(Configuration config) {
+ this.config = config;
+ }
+
+ public Configuration getConfiguration() {
+ return config;
+ }
+
+ // CONFIGS
+
+ public void setMutability(boolean isMutable) {
+ config.setBoolean(MUTABILITY, isMutable);
+ }
+
+ public boolean getMutability() {
+ return config.getBoolean(MUTABILITY, DEFAULT_IS_MUTABLE);
+ }
+
+ public void setBufferTimeout(long timeout) {
+ config.setLong(BUFFER_TIMEOUT, timeout);
+ }
+
+ public long getBufferTimeout() {
+ return config.getLong(BUFFER_TIMEOUT, DEFAULT_TIMEOUT);
+ }
+
+ public void setUserInvokableClass(Class<? extends StreamComponentInvokable> clazz) {
+ config.setClass(USER_FUNCTION, clazz);
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T extends StreamComponentInvokable> Class<? extends T> getUserInvokableClass() {
+ return (Class<? extends T>) config.getClass(USER_FUNCTION, null);
+ }
+
+ public void setUserInvokableObject(StreamComponentInvokable invokableObject) {
+ try {
+ config.setBytes(SERIALIZEDUDF, SerializationUtils.serialize(invokableObject));
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize invokable object "
+ + invokableObject.getClass(), e);
+ }
+ }
+
+ public StreamComponentInvokable getUserInvokableObject() {
+ try {
+ return deserializeObject(config.getBytes(SERIALIZEDUDF, null));
+ } catch (Exception e) {
+ new StreamComponentException("Cannot instantiate user function");
+ }
+ return null;
+ }
+
+ public void setComponentName(String componentName) {
+ config.setString(COMPONENT_NAME, componentName);
+ }
+
+ public String getComponentName() {
+ return config.getString(COMPONENT_NAME, null);
+ }
+
+ public void setFunction(byte[] serializedFunction) {
+ config.setBytes(FUNCTION, serializedFunction);
+ }
+
+ public Object getFunction() {
+ try {
+ return SerializationUtils.deserialize(config
+ .getBytes(FUNCTION, null));
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot deserialize invokable object", e);
+ }
+ }
+
+ public void setFunctionName(String functionName) {
+ config.setString(FUNCTION_NAME, functionName);
+ }
+
+ public String getFunctionName() {
+ return config.getString(FUNCTION_NAME, "");
+ }
+
+ public void setUserDefinedName(String userDefinedName) {
+ if (userDefinedName != null) {
+ config.setString(USER_DEFINED_NAME, userDefinedName);
+ }
+ }
+
+ public void setDirectedEmit(boolean directedEmit) {
+ config.setBoolean(DIRECTED_EMIT, directedEmit);
+ }
+
+ public boolean getDirectedEmit() {
+ return config.getBoolean(DIRECTED_EMIT, false);
+ }
+
+ public void setOutputSelector(byte[] outputSelector) {
+ config.setBytes(OUTPUT_SELECTOR, outputSelector);
+
+ }
+
+ public <T extends Tuple> OutputSelector<T> getOutputSelector() {
+ try {
+ return deserializeObject(config.getBytes(OUTPUT_SELECTOR, null));
+ } catch (Exception e) {
+ throw new StreamComponentException("Cannot deserialize and instantiate OutputSelector",
+ e);
+ }
+ }
+
+ public void setIterationId(String iterationId) {
+ config.setString(ITERATION_ID, iterationId);
+ }
+
+ public String getIterationId() {
+ return config.getString(ITERATION_ID, "iteration-0");
+ }
+
+ public void setNumberOfOutputChannels(int outputIndex, Integer numberOfOutputChannels) {
+ config.setInteger(NUMBER_OF_OUTPUT_CHANNELS + outputIndex, numberOfOutputChannels);
+ }
+
+ public int getNumberOfOutputChannels(int outputIndex) {
+ return config.getInteger(NUMBER_OF_OUTPUT_CHANNELS + outputIndex, 0);
+ }
+
+ public <T extends Tuple> void setPartitioner(int outputIndex,
+ StreamPartitioner<T> partitionerObject) {
+
+ config.setBytes(PARTITIONER_OBJECT + outputIndex,
+ SerializationUtils.serialize(partitionerObject));
+ }
+
+ public <T extends Tuple> StreamPartitioner<T> getPartitioner(int outputIndex)
+ throws ClassNotFoundException, IOException {
+ return deserializeObject(config.getBytes(PARTITIONER_OBJECT + outputIndex,
+ SerializationUtils.serialize(new ShufflePartitioner<T>())));
+ }
+
+ public void setOutputName(int outputIndex, String outputName) {
+ if (outputName != null) {
+ config.setString(OUTPUT_NAME + outputIndex, outputName);
+ }
+ }
+
+ public String getOutputName(int outputIndex) {
+ return config.getString(OUTPUT_NAME + outputIndex, null);
+ }
+
+ public void setNumberOfInputs(int numberOfInputs) {
+ config.setInteger(NUMBER_OF_INPUTS, numberOfInputs);
+ }
+
+ public int getNumberOfInputs() {
+ return config.getInteger(NUMBER_OF_INPUTS, 0);
+ }
+
+ public void setNumberOfOutputs(int numberOfOutputs) {
+ config.setInteger(NUMBER_OF_OUTPUTS, numberOfOutputs);
+ }
+
+ public int getNumberOfOutputs() {
+ return config.getInteger(NUMBER_OF_OUTPUTS, 0);
+ }
+
+ public void setInputType(int inputNumber, Integer inputTypeNumber) {
+ config.setInteger(INPUT_TYPE + inputNumber++, inputTypeNumber);
+ }
+
+ public int getInputType(int inputNumber) {
+ return config.getInteger(INPUT_TYPE + inputNumber, 0);
+ }
+
+ @SuppressWarnings("unchecked")
+ protected static <T> T deserializeObject(byte[] serializedObject) throws IOException,
+ ClassNotFoundException {
+ return (T) SerializationUtils.deserialize(serializedObject);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index 8afbddf..775e722 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -29,20 +29,19 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.operators.util.ReaderIterator;
import org.apache.flink.runtime.plugable.DeserializationDelegate;
import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.StreamConfig;
import org.apache.flink.streaming.api.collector.DirectedStreamCollector;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.collector.StreamCollector;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
-import org.apache.flink.streaming.partitioner.ShufflePartitioner;
import org.apache.flink.streaming.partitioner.StreamPartitioner;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
@@ -55,7 +54,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
protected StreamRecordSerializer<OUT> outTupleSerializer = null;
protected SerializationDelegate<StreamRecord<OUT>> outSerializationDelegate = null;
- protected Configuration configuration;
+ protected StreamConfig configuration;
protected StreamCollector<OUT> collector;
protected int instanceID;
protected String name;
@@ -68,19 +67,13 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
}
protected void initialize() {
- configuration = getTaskConfiguration();
- name = configuration.getString("componentName", "MISSING_COMPONENT_NAME");
+ configuration = new StreamConfig(getTaskConfiguration());
+ name = configuration.getComponentName();
}
protected Collector<OUT> setCollector() {
- if (configuration.getBoolean("directedEmit", false)) {
- OutputSelector<OUT> outputSelector = null;
- try {
- outputSelector = deserializeObject(configuration.getBytes("outputSelector", null));
- } catch (Exception e) {
- throw new StreamComponentException(
- "Cannot deserialize and instantiate OutputSelector", e);
- }
+ if (configuration.getDirectedEmit()) {
+ OutputSelector<OUT> outputSelector = configuration.getOutputSelector();
collector = new DirectedStreamCollector<OUT>(instanceID, outSerializationDelegate,
outputSelector);
@@ -102,7 +95,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
protected void setConfigOutputs(
List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
- int numberOfOutputs = configuration.getInteger("numberOfOutputs", 0);
+ int numberOfOutputs = configuration.getNumberOfOutputs();
for (int i = 0; i < numberOfOutputs; i++) {
setPartitioner(i, outputs);
@@ -111,17 +104,14 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
private void setPartitioner(int outputNumber,
List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
-
- byte[] serializedPartitioner = configuration.getBytes("partitionerObject_" + outputNumber,
- SerializationUtils.serialize((new ShufflePartitioner<OUT>())));
StreamPartitioner<OUT> outputPartitioner = null;
-
+
try {
- outputPartitioner = deserializeObject(serializedPartitioner);
+ outputPartitioner = configuration.getPartitioner(outputNumber);
RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output;
- long bufferTimeout = configuration.getLong("bufferTimeout", 0);
+ long bufferTimeout = configuration.getBufferTimeout();
if (bufferTimeout > 0) {
output = new StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>>(this,
@@ -132,7 +122,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
}
outputs.add(output);
- String outputName = configuration.getString("outputName_" + outputNumber, null);
+ String outputName = configuration.getOutputName(outputNumber);
if (collector != null) {
collector.addOutput(output, outputName);
@@ -143,7 +133,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
+ " with " + outputNumber + " outputs");
}
} catch (Exception e) {
- throw new StreamComponentException("Cannot deserialize "
+ throw new StreamComponentException("Cannot deserialize partitioner "
+ outputPartitioner.getClass().getSimpleName() + " of " + name + " with "
+ outputNumber + " outputs", e);
}
@@ -158,22 +148,9 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
*/
protected StreamComponentInvokable getInvokable(
Class<? extends StreamComponentInvokable> userFunctionClass) {
- StreamComponentInvokable userFunction = null;
-
- byte[] userFunctionSerialized = configuration.getBytes("serializedudf", null);
- this.isMutable = configuration.getBoolean("isMutable", false);
-
- try {
- userFunction = deserializeObject(userFunctionSerialized);
- } catch (ClassNotFoundException e) {
- new StreamComponentException("Cannot instantiate user function: "
- + userFunctionClass.getSimpleName());
- } catch (IOException e) {
- new StreamComponentException("Cannot instantiate user function: "
- + userFunctionClass.getSimpleName());
- }
-
- return userFunction;
+
+ this.isMutable = configuration.getMutability();
+ return configuration.getUserInvokableObject();
}
protected <IN extends Tuple> MutableObjectIterator<StreamRecord<IN>> createInputIterator(
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
index 60a8152..6a9c897 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -19,8 +19,6 @@
package org.apache.flink.streaming.api.streamcomponent;
-import java.io.ByteArrayInputStream;
-import java.io.ObjectInputStream;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
@@ -39,7 +37,6 @@ import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
-import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.util.MutableObjectIterator;
@@ -58,7 +55,7 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private CoInvokable<IN1, IN2, OUT> userFunction;
- private int[] numberOfOutputChannels;
+// private int[] numberOfOutputChannels;
private static int numTasks;
public CoStreamTask() {
@@ -70,21 +67,16 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
}
protected void setSerializers() {
- byte[] operatorBytes = configuration.getBytes("operator", null);
- String operatorName = configuration.getString("operatorName", "");
+ String operatorName = configuration.getFunctionName();
- Object function = null;
+ Object function = configuration.getFunction();
try {
- ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(operatorBytes));
- function = in.readObject();
-
if (operatorName.equals("coMap")) {
setSerializer(function, CoMapFunction.class, 2);
setDeserializers(function, CoMapFunction.class);
} else {
throw new Exception("Wrong operator name!");
}
-
} catch (Exception e) {
throw new StreamComponentException(e);
}
@@ -119,10 +111,10 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
setConfigOutputs(outputs);
- numberOfOutputChannels = new int[outputs.size()];
- for (int i = 0; i < numberOfOutputChannels.length; i++) {
- numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
- }
+// numberOfOutputChannels = new int[outputs.size()];
+// for (int i = 0; i < numberOfOutputChannels.length; i++) {
+// numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
+// }
setInvokable();
}
@@ -131,21 +123,21 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
@Override
protected void setInvokable() {
// Default value is a CoMapInvokable
- Class<? extends CoInvokable> userFunctionClass = configuration.getClass("userfunction",
- CoMapInvokable.class, CoInvokable.class);
+ Class<? extends CoInvokable> userFunctionClass = configuration.getUserInvokableClass();
+
userFunction = (CoInvokable<IN1, IN2, OUT>) getInvokable(userFunctionClass);
userFunction.initialize(collector, inputIter1, inTupleSerializer1, inputIter2,
inTupleSerializer2, isMutable);
}
protected void setConfigInputs() throws StreamComponentException {
- int numberOfInputs = configuration.getInteger("numberOfInputs", 0);
+ int numberOfInputs = configuration.getNumberOfInputs();
ArrayList<MutableRecordReader<IOReadableWritable>> inputList1 = new ArrayList<MutableRecordReader<IOReadableWritable>>();
ArrayList<MutableRecordReader<IOReadableWritable>> inputList2 = new ArrayList<MutableRecordReader<IOReadableWritable>>();
for (int i = 0; i < numberOfInputs; i++) {
- int inputType = configuration.getInteger("inputType_" + i, 0);
+ int inputType = configuration.getInputType(i);
switch (inputType) {
case 1:
inputList1.add(new MutableRecordReader<IOReadableWritable>(this));
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
index b49620f..86b26ff 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
@@ -19,9 +19,6 @@
package org.apache.flink.streaming.api.streamcomponent;
-import java.io.ByteArrayInputStream;
-import java.io.ObjectInputStream;
-
import org.apache.flink.api.common.functions.AbstractFunction;
import org.apache.flink.api.java.functions.FilterFunction;
import org.apache.flink.api.java.functions.FlatMapFunction;
@@ -47,14 +44,10 @@ public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT e
protected StreamRecordSerializer<IN> inTupleSerializer = null;
protected void setSerializers() {
- byte[] operatorBytes = configuration.getBytes("operator", null);
- String operatorName = configuration.getString("operatorName", "");
+ String operatorName = configuration.getFunctionName();
- Object function = null;
+ Object function = configuration.getFunction();
try {
- ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(operatorBytes));
- function = in.readObject();
-
if (operatorName.equals("flatMap")) {
setSerializerDeserializer(function, FlatMapFunction.class);
} else if (operatorName.equals("map")) {
@@ -110,7 +103,7 @@ public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT e
@SuppressWarnings("unchecked")
protected MutableReader<IOReadableWritable> getConfigInputs() throws StreamComponentException {
- int numberOfInputs = configuration.getInteger("numberOfInputs", 0);
+ int numberOfInputs = configuration.getNumberOfInputs();
if (numberOfInputs < 2) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
index 1b25285..224fdfb 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
@@ -53,7 +53,7 @@ public class StreamIterationSink<IN extends Tuple> extends SingleInputAbstractSt
setSinkSerializer();
inputs = getConfigInputs();
inputIter = createInputIterator(inputs, inTupleSerializer);
- iterationId = configuration.getString("iteration-id", "iteration-0");
+ iterationId = configuration.getIterationId();
dataChannel = BlockingQueueBroker.instance().get(iterationId);
} catch (Exception e) {
throw new StreamComponentException(String.format(
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
index f880470..37e8e0f 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
@@ -38,7 +38,7 @@ public class StreamIterationSource<OUT extends Tuple> extends SingleInputAbstrac
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private static int numSources;
- private int[] numberOfOutputChannels;
+// private int[] numberOfOutputChannels;
private String iterationId;
@SuppressWarnings("rawtypes")
private BlockingQueue<StreamRecord> dataChannel;
@@ -63,12 +63,12 @@ public class StreamIterationSource<OUT extends Tuple> extends SingleInputAbstrac
throw new StreamComponentException("Cannot register outputs", e);
}
- numberOfOutputChannels = new int[outputs.size()];
- for (int i = 0; i < numberOfOutputChannels.length; i++) {
- numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
- }
+// numberOfOutputChannels = new int[outputs.size()];
+// for (int i = 0; i < numberOfOutputChannels.length; i++) {
+// numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
+// }
- iterationId = configuration.getString("iteration-id", "iteration-0");
+ iterationId = configuration.getIterationId();
try {
BlockingQueueBroker.instance().handIn(iterationId, dataChannel);
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
index 5e3457c..8cbe0ea 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
@@ -61,8 +61,8 @@ public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamCompo
@SuppressWarnings({ "rawtypes", "unchecked" })
@Override
protected void setInvokable() {
- Class<? extends SinkInvokable> userFunctionClass = configuration.getClass("userfunction",
- SinkInvokable.class, SinkInvokable.class);
+ Class<? extends SinkInvokable> userFunctionClass = configuration.getUserInvokableClass();
+
userFunction = (SinkInvokable<IN>) getInvokable(userFunctionClass);
userFunction.initialize(collector, inputIter, inTupleSerializer, isMutable);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
index 12c9ba3..c4f38ce 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
@@ -37,7 +37,7 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private UserSourceInvokable<OUT> userFunction;
private static int numSources;
- private int[] numberOfOutputChannels;
+// private int[] numberOfOutputChannels;
public StreamSource() {
@@ -60,10 +60,10 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
+ getClass().getSimpleName(), e);
}
- numberOfOutputChannels = new int[outputs.size()];
- for (int i = 0; i < numberOfOutputChannels.length; i++) {
- numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
- }
+// numberOfOutputChannels = new int[outputs.size()];
+// for (int i = 0; i < numberOfOutputChannels.length; i++) {
+// numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
+// }
setInvokable();
}
@@ -72,8 +72,8 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
@Override
protected void setInvokable() {
// Default value is a TaskInvokable even if it was called from a source
- Class<? extends UserSourceInvokable> userFunctionClass = configuration.getClass(
- "userfunction", UserSourceInvokable.class, UserSourceInvokable.class);
+ Class<? extends UserSourceInvokable> userFunctionClass = configuration.getUserInvokableClass();
+// .getClass("userfunction", UserSourceInvokable.class, UserSourceInvokable.class);
userFunction = (UserSourceInvokable<OUT>) getInvokable(userFunctionClass);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
index 5032446..12e064f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
@@ -43,7 +43,7 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
MutableObjectIterator<StreamRecord<IN>> inputIter;
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private StreamRecordInvokable<IN, OUT> userFunction;
- private int[] numberOfOutputChannels;
+// private int[] numberOfOutputChannels;
private static int numTasks;
public StreamTask() {
@@ -65,10 +65,10 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
inputIter = createInputIterator(inputs, inTupleSerializer);
- numberOfOutputChannels = new int[outputs.size()];
- for (int i = 0; i < numberOfOutputChannels.length; i++) {
- numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
- }
+// numberOfOutputChannels = new int[outputs.size()];
+// for (int i = 0; i < numberOfOutputChannels.length; i++) {
+// numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
+// }
setInvokable();
}
@@ -77,8 +77,7 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
@Override
protected void setInvokable() {
// Default value is a TaskInvokable even if it was called from a source
- Class<? extends UserTaskInvokable> userFunctionClass = configuration.getClass(
- "userfunction", UserTaskInvokable.class, UserTaskInvokable.class);
+ Class<? extends UserTaskInvokable> userFunctionClass = configuration.getUserInvokableClass();
userFunction = (UserTaskInvokable<IN, OUT>) getInvokable(userFunctionClass);
userFunction.initialize(collector, inputIter, inTupleSerializer, isMutable);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamWindowTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamWindowTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamWindowTask.java
deleted file mode 100644
index c044cc3..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamWindowTask.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.streamcomponent;
-
-import java.util.ArrayList;
-
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.state.SlidingWindowState;
-import org.apache.flink.streaming.state.StateManager;
-import org.apache.flink.util.Collector;
-
-public class StreamWindowTask<InTuple extends Tuple, OutTuple extends Tuple>
- extends FlatMapFunction<InTuple, OutTuple> {
- private static final long serialVersionUID = 1L;
-
- private int computeGranularity;
- private int windowFieldId;
-
- private ArrayList<InTuple> tempTupleArray;
- private SlidingWindowState<InTuple> window;
- private long initTimestamp = -1;
- private long nextTimestamp = -1;
-
- protected StateManager checkpointer = new StateManager("object.out", 1000);
-
- public StreamWindowTask(int windowSize, int slidingStep,
- int computeGranularity, int windowFieldId) {
- this.computeGranularity = computeGranularity;
- this.windowFieldId = windowFieldId;
- window = new SlidingWindowState<InTuple>(windowSize, slidingStep,
- computeGranularity);
- checkpointer.registerState(window);
- Thread t = new Thread(checkpointer);
- t.start();
- }
-
- protected void incrementCompute(ArrayList<InTuple> tupleArray) {
- }
-
- protected void decrementCompute(ArrayList<InTuple> tupleArray) {
- }
-
- protected void produceOutput(long progress, Collector<OutTuple> out) {
- }
-
- @Override
- public void flatMap(InTuple value, Collector<OutTuple> out)
- throws Exception {
- long progress = (Long) value.getField(windowFieldId);
- if (initTimestamp == -1) {
- initTimestamp = progress;
- nextTimestamp = initTimestamp + computeGranularity;
- tempTupleArray = new ArrayList<InTuple>();
- } else {
- if (progress > nextTimestamp) {
- if (window.isFull()) {
- ArrayList<InTuple> expiredTupleArray = window.popFront();
- incrementCompute(tempTupleArray);
- decrementCompute(expiredTupleArray);
- window.pushBack(tempTupleArray);
- if (window.isEmittable()) {
- produceOutput(progress, out);
- }
- } else {
- incrementCompute(tempTupleArray);
- window.pushBack(tempTupleArray);
- if (window.isFull()) {
- produceOutput(progress, out);
- }
- }
- initTimestamp = nextTimestamp;
- nextTimestamp = initTimestamp + computeGranularity;
- tempTupleArray = new ArrayList<InTuple>();
- }
- tempTupleArray.add(value);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumAggregate.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumAggregate.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumAggregate.java
deleted file mode 100644
index 12ada98..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumAggregate.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.examples.window.sum;
-
-import java.util.ArrayList;
-
-import org.apache.flink.streaming.api.streamcomponent.StreamWindowTask;
-import org.apache.flink.streaming.state.TableState;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-
-public class WindowSumAggregate extends
- StreamWindowTask<Tuple2<Integer, Long>, Tuple2<Integer, Long>> {
- private static final long serialVersionUID = -2832409561059237150L;
- private TableState<String, Integer> sum;
- private Tuple2<Integer, Long> outTuple = new Tuple2<Integer, Long>();
-
-
- public WindowSumAggregate(int windowSize, int slidingStep,
- int computeGranularity, int windowFieldId) {
- super(windowSize, slidingStep, computeGranularity, windowFieldId);
- sum = new TableState<String, Integer>();
- sum.put("sum", 0);
- checkpointer.registerState(sum);
- }
-
- @Override
- protected void incrementCompute(ArrayList<Tuple2<Integer, Long>> tupleArray) {
- for (int i = 0; i < tupleArray.size(); ++i) {
- int number = tupleArray.get(i).f0;
- sum.put("sum", sum.get("sum") + number);
- }
- }
-
- @Override
- protected void decrementCompute(ArrayList<Tuple2<Integer, Long>> tupleArray) {
- for (int i = 0; i < tupleArray.size(); ++i) {
- int number = tupleArray.get(i).f0;
- sum.put("sum", sum.get("sum") - number);
- }
- }
-
- @Override
- protected void produceOutput(long progress, Collector<Tuple2<Integer, Long>> out){
- outTuple.f0 = sum.get("sum");
- outTuple.f1 = progress;
- out.collect(outTuple);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumLocal.java
deleted file mode 100644
index c8390f0..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumLocal.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.examples.window.sum;
-
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-
-public class WindowSumLocal {
-
- private static final int PARALLELISM = 1;
- private static final int SOURCE_PARALLELISM = 1;
-
- public static void main(String[] args) {
- StreamExecutionEnvironment env = StreamExecutionEnvironment
- .createLocalEnvironment(PARALLELISM);
-
- @SuppressWarnings("unused")
- DataStream<Tuple2<Integer, Long>> dataStream = env
- .addSource(new WindowSumSource(), SOURCE_PARALLELISM).map(new WindowSumMultiple())
- .flatMap(new WindowSumAggregate(100, 20, 10, 1)).addSink(new WindowSumSink());
-
- env.execute();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumMultiple.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumMultiple.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumMultiple.java
deleted file mode 100644
index 8d4ac0c..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumMultiple.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.examples.window.sum;
-
-import org.apache.flink.api.java.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-
-public class WindowSumMultiple extends MapFunction<Tuple2<Integer, Long>, Tuple2<Integer, Long>> {
- private static final long serialVersionUID = 1L;
-
- private Tuple2<Integer, Long> outTuple = new Tuple2<Integer, Long>();
-
- @Override
- public Tuple2<Integer, Long> map(Tuple2<Integer, Long> inTuple) throws Exception {
- outTuple.f0 = inTuple.f0 * 2;
- outTuple.f1 = inTuple.f1;
- return outTuple;
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSink.java
deleted file mode 100644
index 9c4547a..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSink.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.examples.window.sum;
-
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-
-public class WindowSumSink extends SinkFunction<Tuple2<Integer, Long>> {
- private static final long serialVersionUID = 1L;
- @Override
- public void invoke(Tuple2<Integer, Long> inTuple) {
- System.out.println(inTuple);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSource.java
deleted file mode 100644
index 12eda38..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/sum/WindowSumSource.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.examples.window.sum;
-
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-
-public class WindowSumSource extends SourceFunction<Tuple2<Integer, Long>> {
- private static final long serialVersionUID = 1L;
-
- private Tuple2<Integer, Long> outRecord = new Tuple2<Integer, Long>();
- private Long timestamp = 0L;
-
- @Override
- public void invoke(Collector<Tuple2<Integer, Long>> collector) throws Exception {
- for (int i = 0; i < 1000; ++i) {
- outRecord.f0 = i;
- outRecord.f1 = timestamp;
- collector.collect(outRecord);
- timestamp++;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountCounter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountCounter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountCounter.java
deleted file mode 100644
index dd99271..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountCounter.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.examples.window.wordcount;
-
-import java.util.ArrayList;
-
-import org.apache.flink.streaming.api.streamcomponent.StreamWindowTask;
-import org.apache.flink.streaming.state.TableState;
-import org.apache.flink.streaming.state.TableStateIterator;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.util.Collector;
-
-public class WindowWordCountCounter extends
- StreamWindowTask<Tuple2<String, Long>, Tuple3<String, Integer, Long>> {
- private static final long serialVersionUID = 1L;
-
- private Tuple3<String, Integer, Long> outTuple = new Tuple3<String, Integer, Long>();
- private TableState<String, Integer> wordCounts;
-
- public WindowWordCountCounter(int windowSize, int slidingStep,
- int computeGranularity, int windowFieldId) {
- super(windowSize, slidingStep, computeGranularity, windowFieldId);
- wordCounts = new TableState<String, Integer>();
- }
-
- @Override
- protected void incrementCompute(ArrayList<Tuple2<String, Long>> tupleArray) {
- for (int i = 0; i < tupleArray.size(); ++i) {
- String word = tupleArray.get(i).f0;
- if (wordCounts.containsKey(word)) {
- int count = wordCounts.get(word) + 1;
- wordCounts.put(word, count);
- } else {
- wordCounts.put(word, 1);
- }
- }
- }
-
- @Override
- protected void decrementCompute(ArrayList<Tuple2<String, Long>> tupleArray) {
- for (int i = 0; i < tupleArray.size(); ++i) {
- String word = tupleArray.get(i).f0;
- int count = wordCounts.get(word) - 1;
- if (count == 0) {
- wordCounts.delete(word);
- } else {
- wordCounts.put(word, count);
- }
- }
- }
-
- @Override
- protected void produceOutput(long progress, Collector<Tuple3<String, Integer, Long>> out) {
- TableStateIterator<String, Integer> iterator = wordCounts.getIterator();
- while (iterator.hasNext()) {
- Tuple2<String, Integer> tuple = iterator.next();
- outTuple.f0 = tuple.f0;
- outTuple.f1 = tuple.f1;
- outTuple.f2 = progress;
- out.collect(outTuple);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountLocal.java
deleted file mode 100644
index 2ef5af9..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountLocal.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.examples.window.wordcount;
-
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.TestDataUtil;
-
-import org.apache.flink.api.java.tuple.Tuple3;
-
-public class WindowWordCountLocal {
-
- private static final int PARALLELISM = 1;
-
- // This example will count the occurrence of each word in the input file with a sliding window.
-
- public static void main(String[] args) {
-
- TestDataUtil.downloadIfNotExists("hamlet.txt");
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(PARALLELISM);
-
- @SuppressWarnings("unused")
- DataStream<Tuple3<String, Integer, Long>> dataStream = env
- .readTextStream("src/test/resources/testdata/hamlet.txt")
- .flatMap(new WindowWordCountSplitter())
- .partitionBy(0)
- .flatMap(new WindowWordCountCounter(10, 2, 1, 1))
- .addSink(new WindowWordCountSink());
-
- env.execute();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSink.java
deleted file mode 100644
index f8c009f..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSink.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.examples.window.wordcount;
-
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.api.java.tuple.Tuple3;
-
-public class WindowWordCountSink extends SinkFunction<Tuple3<String, Integer, Long>> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(Tuple3<String, Integer, Long> inTuple) {
- System.out.println(inTuple);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/330d8fd5/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSplitter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSplitter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSplitter.java
deleted file mode 100644
index 19a533e..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/wordcount/WindowWordCountSplitter.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.examples.window.wordcount;
-
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-
-public class WindowWordCountSplitter extends FlatMapFunction<Tuple1<String>, Tuple2<String, Long>> {
- private static final long serialVersionUID = 1L;
-
- private String[] words = new String[] {};
- private Long timestamp = 0L;
- private Tuple2<String, Long> outTuple = new Tuple2<String, Long>();
-
- // Splits the lines according to the spaces. And adds the line's timestamp to them.
- @Override
- public void flatMap(Tuple1<String> inTuple, Collector<Tuple2<String, Long>> out) throws Exception {
-
- words=inTuple.f0.split(" ");
- timestamp=System.currentTimeMillis();
- for(String word : words){
- outTuple.f0 = word;
- outTuple.f1 = timestamp;
- out.collect(outTuple);
- }
- }
-}
\ No newline at end of file
[02/51] [abbrv] git commit: [streaming] Package refactor & cleanup
Posted by se...@apache.org.
[streaming] Package refactor & cleanup
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/60f632a3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/60f632a3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/60f632a3
Branch: refs/heads/master
Commit: 60f632a305f7019b638088f0a6d88ed2b42b267f
Parents: c79b48b
Author: mbalassi <ba...@gmail.com>
Authored: Fri Jul 18 12:36:04 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:14:12 2014 +0200
----------------------------------------------------------------------
.../streaming/connectors/kafka/KafkaSink.java | 2 +-
.../streaming/connectors/kafka/KafkaSource.java | 3 +-
.../connectors/kafka/KafkaTopology.java | 3 +-
.../streaming/connectors/rabbitmq/RMQSink.java | 2 +-
.../connectors/rabbitmq/RMQSource.java | 2 +-
.../streaming/connectors/rabbitmq/RMQTest.java | 2 +-
.../apache/flink/streaming/api/DataStream.java | 6 +-
.../api/StreamExecutionEnvironment.java | 22 ++--
.../api/function/FileSourceFunction.java | 53 ----------
.../api/function/FileStreamFunction.java | 54 ----------
.../api/function/FromElementsFunction.java | 50 ---------
.../api/function/GenSequenceFunction.java | 50 ---------
.../api/function/PrintSinkFunction.java | 39 -------
.../streaming/api/function/SinkFunction.java | 33 ------
.../streaming/api/function/SourceFunction.java | 30 ------
.../streaming/api/function/WriteFormat.java | 47 ---------
.../api/function/WriteFormatAsCsv.java | 53 ----------
.../api/function/WriteFormatAsText.java | 51 ----------
.../api/function/WriteSinkFunction.java | 101 -------------------
.../function/WriteSinkFunctionByBatches.java | 52 ----------
.../api/function/WriteSinkFunctionByMillis.java | 54 ----------
.../api/function/sink/PrintSinkFunction.java | 39 +++++++
.../api/function/sink/SinkFunction.java | 33 ++++++
.../api/function/sink/WriteFormat.java | 47 +++++++++
.../api/function/sink/WriteFormatAsCsv.java | 53 ++++++++++
.../api/function/sink/WriteFormatAsText.java | 51 ++++++++++
.../api/function/sink/WriteSinkFunction.java | 101 +++++++++++++++++++
.../sink/WriteSinkFunctionByBatches.java | 52 ++++++++++
.../sink/WriteSinkFunctionByMillis.java | 54 ++++++++++
.../api/function/source/FileSourceFunction.java | 53 ++++++++++
.../api/function/source/FileStreamFunction.java | 54 ++++++++++
.../function/source/FromElementsFunction.java | 50 +++++++++
.../function/source/GenSequenceFunction.java | 50 +++++++++
.../api/function/source/SourceFunction.java | 30 ++++++
.../streaming/api/invokable/SinkInvokable.java | 3 +-
.../AbstractStreamComponent.java | 2 +-
.../apache/flink/streaming/api/IterateTest.java | 2 +-
.../flink/streaming/api/WriteAsCsvTest.java | 21 +++-
.../flink/streaming/api/WriteAsTextTest.java | 6 +-
.../api/collector/DirectedOutputTest.java | 2 +-
.../api/invokable/operator/BatchReduceTest.java | 4 +-
.../api/invokable/operator/FlatMapTest.java | 2 +-
.../api/invokable/operator/MapTest.java | 4 +-
.../streamcomponent/StreamComponentTest.java | 5 +-
.../examples/basictopology/BasicTopology.java | 3 +-
.../examples/cellinfo/CellInfoLocal.java | 3 +-
.../CollaborativeFilteringSink.java | 3 +-
.../CollaborativeFilteringSource.java | 3 +-
.../examples/iterative/kmeans/KMeansSink.java | 3 +-
.../examples/iterative/kmeans/KMeansSource.java | 3 +-
.../iterative/pagerank/PageRankSink.java | 3 +-
.../iterative/pagerank/PageRankSource.java | 2 +-
.../examples/iterative/sssp/SSSPSink.java | 3 +-
.../examples/iterative/sssp/SSSPSource.java | 3 +-
.../flink/streaming/examples/join/JoinSink.java | 3 +-
.../streaming/examples/join/JoinSourceOne.java | 3 +-
.../streaming/examples/join/JoinSourceTwo.java | 3 +-
.../ml/IncrementalLearningSkeleton.java | 3 +-
.../streaming/examples/ml/IncrementalOLS.java | 3 +-
.../window/join/WindowJoinSourceOne.java | 3 +-
.../window/join/WindowJoinSourceTwo.java | 3 +-
.../examples/window/sum/WindowSumSink.java | 3 +-
.../examples/window/sum/WindowSumSource.java | 3 +-
.../window/wordcount/WindowWordCountSink.java | 3 +-
64 files changed, 742 insertions(+), 744 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
index 42795b4..eeac961 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
@@ -26,7 +26,7 @@ import kafka.producer.KeyedMessage;
import kafka.producer.ProducerConfig;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.function.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
public abstract class KafkaSink<IN extends Tuple, OUT> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
index 2f9b613..d34b6c3 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
@@ -24,12 +24,13 @@ import java.util.List;
import java.util.Map;
import java.util.Properties;
-import org.apache.flink.streaming.api.function.SourceFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import kafka.consumer.ConsumerConfig;
import kafka.consumer.ConsumerIterator;
import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
+
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
index 4c0b636..1a77aee 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
@@ -21,8 +21,7 @@ package org.apache.flink.streaming.connectors.kafka;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.SourceFunction;
-
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index 6317b89..e6e8de5 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -24,7 +24,7 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.function.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index c3d121b..4b197e3 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -24,7 +24,7 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.function.SourceFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
import com.rabbitmq.client.Channel;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java
index 673c560..c6a43f2 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTest.java
@@ -24,7 +24,7 @@ import java.util.HashSet;
import java.util.Set;
import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.function.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.junit.Test;
public class RMQTest {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index a94834a..6e16a85 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -29,9 +29,9 @@ import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.StreamExecutionEnvironment.ConnectionType;
import org.apache.flink.streaming.api.collector.OutputSelector;
-import org.apache.flink.streaming.api.function.SinkFunction;
-import org.apache.flink.streaming.api.function.WriteFormatAsCsv;
-import org.apache.flink.streaming.api.function.WriteFormatAsText;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
+import org.apache.flink.streaming.api.function.sink.WriteFormatAsText;
import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index 86c54ef..87cd154 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -28,17 +28,17 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.collector.OutputSelector;
-import org.apache.flink.streaming.api.function.FileSourceFunction;
-import org.apache.flink.streaming.api.function.FileStreamFunction;
-import org.apache.flink.streaming.api.function.FromElementsFunction;
-import org.apache.flink.streaming.api.function.GenSequenceFunction;
-import org.apache.flink.streaming.api.function.PrintSinkFunction;
-import org.apache.flink.streaming.api.function.SinkFunction;
-import org.apache.flink.streaming.api.function.SourceFunction;
-import org.apache.flink.streaming.api.function.WriteFormatAsCsv;
-import org.apache.flink.streaming.api.function.WriteFormatAsText;
-import org.apache.flink.streaming.api.function.WriteSinkFunctionByBatches;
-import org.apache.flink.streaming.api.function.WriteSinkFunctionByMillis;
+import org.apache.flink.streaming.api.function.sink.PrintSinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
+import org.apache.flink.streaming.api.function.sink.WriteFormatAsText;
+import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByBatches;
+import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByMillis;
+import org.apache.flink.streaming.api.function.source.FileSourceFunction;
+import org.apache.flink.streaming.api.function.source.FileStreamFunction;
+import org.apache.flink.streaming.api.function.source.FromElementsFunction;
+import org.apache.flink.streaming.api.function.source.GenSequenceFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.api.invokable.SinkInvokable;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FileSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FileSourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FileSourceFunction.java
deleted file mode 100644
index 0c98117..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FileSourceFunction.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.function;
-
-import java.io.BufferedReader;
-import java.io.FileReader;
-import java.io.IOException;
-
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.util.Collector;
-
-public class FileSourceFunction extends SourceFunction<Tuple1<String>> {
- private static final long serialVersionUID = 1L;
-
- private final String path;
- private Tuple1<String> outTuple = new Tuple1<String>();
-
- public FileSourceFunction(String path) {
- this.path = path;
- }
-
- @Override
- public void invoke(Collector<Tuple1<String>> collector) throws IOException {
- BufferedReader br = new BufferedReader(new FileReader(path));
- String line = br.readLine();
- while (line != null) {
- if (line != "") {
- outTuple.f0 = line;
- collector.collect(outTuple);
- }
- line = br.readLine();
- }
- br.close();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FileStreamFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FileStreamFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FileStreamFunction.java
deleted file mode 100644
index 15e685e..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FileStreamFunction.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.function;
-
-import java.io.BufferedReader;
-import java.io.FileReader;
-import java.io.IOException;
-
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.util.Collector;
-
-public class FileStreamFunction extends SourceFunction<Tuple1<String>>{
- private static final long serialVersionUID = 1L;
-
- private final String path;
- private Tuple1<String> outTuple = new Tuple1<String>();
-
- public FileStreamFunction(String path) {
- this.path = path;
- }
-
- @Override
- public void invoke(Collector<Tuple1<String>> collector) throws IOException {
- while(true){
- BufferedReader br = new BufferedReader(new FileReader(path));
- String line = br.readLine();
- while (line != null) {
- if (line != "") {
- outTuple.f0 = line;
- collector.collect(outTuple);
- }
- line = br.readLine();
- }
- br.close();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FromElementsFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FromElementsFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FromElementsFunction.java
deleted file mode 100755
index 6e6d194..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/FromElementsFunction.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.function;
-
-import java.util.Arrays;
-import java.util.Collection;
-
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.util.Collector;
-
-public class FromElementsFunction<T> extends SourceFunction<Tuple1<T>> {
- private static final long serialVersionUID = 1L;
-
- Iterable<T> iterable;
- Tuple1<T> outTuple = new Tuple1<T>();
-
- public FromElementsFunction(T... elements) {
- this.iterable = Arrays.asList(elements);
- }
-
- public FromElementsFunction(Collection<T> elements) {
- this.iterable = elements;
- }
-
- @Override
- public void invoke(Collector<Tuple1<T>> collector) throws Exception {
- for (T element : iterable) {
- outTuple.f0 = element;
- collector.collect(outTuple);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/GenSequenceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/GenSequenceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/GenSequenceFunction.java
deleted file mode 100755
index 08505c3..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/GenSequenceFunction.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.function;
-
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.util.Collector;
-
-/**
- * Source Function used to generate the number sequence
- *
- */
-public class GenSequenceFunction extends SourceFunction<Tuple1<Long>> {
-
- private static final long serialVersionUID = 1L;
-
- long from;
- long to;
- Tuple1<Long> outTuple = new Tuple1<Long>();
-
- public GenSequenceFunction(long from, long to) {
- this.from = from;
- this.to = to;
- }
-
- @Override
- public void invoke(Collector<Tuple1<Long>> collector) throws Exception {
- for (long i = from; i <= to; i++) {
- outTuple.f0 = i;
- collector.collect(outTuple);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/PrintSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/PrintSinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/PrintSinkFunction.java
deleted file mode 100755
index 18e34e8..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/PrintSinkFunction.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.function;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Dummy implementation of the SinkFunction writing every tuple to the standard
- * output. Used for print.
- *
- * @param <IN>
- * Input tuple type
- */
-public class PrintSinkFunction<IN extends Tuple> extends SinkFunction<IN> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void invoke(IN tuple) {
- System.out.println(tuple);
- }
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/SinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/SinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/SinkFunction.java
deleted file mode 100644
index 2136ef3..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/SinkFunction.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.function;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.AbstractFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-
-public abstract class SinkFunction<IN extends Tuple> extends AbstractFunction implements Serializable {
-
- private static final long serialVersionUID = 1L;
-
- public abstract void invoke(IN tuple);
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/SourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/SourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/SourceFunction.java
deleted file mode 100755
index 0616605..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/SourceFunction.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.function;
-
-import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-public abstract class SourceFunction<OUT extends Tuple> extends UserSourceInvokable<OUT> {
-
- private static final long serialVersionUID = 1L;
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormat.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormat.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormat.java
deleted file mode 100644
index d5f63d8..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormat.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/***********************************************************************************************************************
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- **********************************************************************************************************************/
-
-package org.apache.flink.streaming.api.function;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Abstract class for formatting the output of the writeAsText and writeAsCsv
- * functions.
- *
- * @param <IN>
- * Input tuple type
- */
-public abstract class WriteFormat<IN extends Tuple> implements Serializable {
- private static final long serialVersionUID = 1L;
-
- /**
- * Writes the contents of tupleList to the file specified by path.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param tupleList
- * is the list of tuples to be written
- */
- protected abstract void write(String path, ArrayList<IN> tupleList);
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormatAsCsv.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormatAsCsv.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormatAsCsv.java
deleted file mode 100644
index 95284b9..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormatAsCsv.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/***********************************************************************************************************************
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- **********************************************************************************************************************/
-
-package org.apache.flink.streaming.api.function;
-
-import java.io.BufferedWriter;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Writes tuples in csv format.
- *
- * @param <IN>
- * Input tuple type
- */
-public class WriteFormatAsCsv<IN extends Tuple> extends WriteFormat<IN> {
- private static final long serialVersionUID = 1L;
-
- @Override
- protected void write(String path, ArrayList<IN> tupleList) {
- try {
- PrintWriter outStream = new PrintWriter(new BufferedWriter(new FileWriter(path, true)));
- for (IN tupleToWrite : tupleList) {
- outStream.println(tupleToWrite.toString().substring(1,
- tupleToWrite.toString().length() - 1));
- }
- outStream.close();
- } catch (IOException e) {
- throw new RuntimeException("Exception occured while writing file " + path, e);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormatAsText.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormatAsText.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormatAsText.java
deleted file mode 100644
index 7071dd6..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteFormatAsText.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/***********************************************************************************************************************
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- **********************************************************************************************************************/
-
-package org.apache.flink.streaming.api.function;
-
-import java.io.BufferedWriter;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Writes tuples in text format.
- *
- * @param <IN>
- * Input tuple type
- */
-public class WriteFormatAsText<IN extends Tuple> extends WriteFormat<IN> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void write(String path, ArrayList<IN> tupleList) {
- try {
- PrintWriter outStream = new PrintWriter(new BufferedWriter(new FileWriter(path, true)));
- for (IN tupleToWrite : tupleList) {
- outStream.println(tupleToWrite);
- }
- outStream.close();
- } catch (IOException e) {
- throw new RuntimeException("Exception occured while writing file " + path, e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunction.java
deleted file mode 100644
index 9ef476d..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunction.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/***********************************************************************************************************************
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- **********************************************************************************************************************/
-
-package org.apache.flink.streaming.api.function;
-
-import java.io.FileNotFoundException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Simple implementation of the SinkFunction writing tuples as simple text to
- * the file specified by path. Tuples are collected to a list and written to the
- * file periodically. The file specified by path is created if it does not
- * exist, cleared if it exists before the writing.
- *
- * @param <IN>
- * Input tuple type
- */
-public abstract class WriteSinkFunction<IN extends Tuple> extends SinkFunction<IN> {
- private static final long serialVersionUID = 1L;
-
- protected final String path;
- protected ArrayList<IN> tupleList = new ArrayList<IN>();
- protected final IN endTuple;
- protected WriteFormat<IN> format;
-
- public WriteSinkFunction(String path, WriteFormat<IN> format, IN endTuple) {
- this.path = path;
- this.format = format;
- this.endTuple = endTuple;
- cleanFile(path);
- }
-
- /**
- * Creates target file if it does not exist, cleans it if it exists.
- *
- * @param path
- * is the path to the location where the tuples are written
- */
- protected void cleanFile(String path) {
- try {
- PrintWriter writer;
- writer = new PrintWriter(path);
- writer.print("");
- writer.close();
- } catch (FileNotFoundException e) {
- throw new RuntimeException("File not found " + path, e);
- }
- }
-
- /**
- * Condition for writing the contents of tupleList and clearing it.
- *
- * @return value of the updating condition
- */
- protected abstract boolean updateCondition();
-
- /**
- * Statements to be executed after writing a batch goes here.
- */
- protected abstract void resetParameters();
-
- /**
- * Implementation of the invoke method of the SinkFunction class. Collects
- * the incoming tuples in tupleList and appends the list to the end of the
- * target file if updateCondition() is true or the current tuple is the
- * endTuple.
- */
- @Override
- public void invoke(IN tuple) {
- if (!tuple.equals(endTuple)) {
- tupleList.add(tuple);
- if (updateCondition()) {
- format.write(path, tupleList);
- resetParameters();
- }
- } else {
- format.write(path, tupleList);
- resetParameters();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunctionByBatches.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunctionByBatches.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunctionByBatches.java
deleted file mode 100644
index b3dea66..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunctionByBatches.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/***********************************************************************************************************************
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- **********************************************************************************************************************/
-
-package org.apache.flink.streaming.api.function;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Implementation of WriteSinkFunction. Writes tuples to file in equally sized
- * batches.
- *
- * @param <IN>
- * Input tuple type
- */
-public class WriteSinkFunctionByBatches<IN extends Tuple> extends WriteSinkFunction<IN> {
- private static final long serialVersionUID = 1L;
-
- private final int batchSize;
-
- public WriteSinkFunctionByBatches(String path, WriteFormat<IN> format, int batchSize,
- IN endTuple) {
- super(path, format, endTuple);
- this.batchSize = batchSize;
- }
-
- @Override
- protected boolean updateCondition() {
- return tupleList.size() >= batchSize;
- }
-
- @Override
- protected void resetParameters() {
- tupleList.clear();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunctionByMillis.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunctionByMillis.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunctionByMillis.java
deleted file mode 100644
index b843294..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/WriteSinkFunctionByMillis.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/***********************************************************************************************************************
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- **********************************************************************************************************************/
-
-package org.apache.flink.streaming.api.function;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Implementation of WriteSinkFunction. Writes tuples to file in every millis
- * milliseconds.
- *
- * @param <IN>
- * Input tuple type
- */
-public class WriteSinkFunctionByMillis<IN extends Tuple> extends WriteSinkFunction<IN> {
- private static final long serialVersionUID = 1L;
-
- private final long millis;
- private long lastTime;
-
- public WriteSinkFunctionByMillis(String path, WriteFormat<IN> format, long millis, IN endTuple) {
- super(path, format, endTuple);
- this.millis = millis;
- lastTime = System.currentTimeMillis();
- }
-
- @Override
- protected boolean updateCondition() {
- return System.currentTimeMillis() - lastTime >= millis;
- }
-
- @Override
- protected void resetParameters() {
- tupleList.clear();
- lastTime = System.currentTimeMillis();
- }
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
new file mode 100755
index 0000000..7918e48
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
@@ -0,0 +1,39 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.sink;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Dummy implementation of the SinkFunction writing every tuple to the standard
+ * output. Used for print.
+ *
+ * @param <IN>
+ * Input tuple type
+ */
+public class PrintSinkFunction<IN extends Tuple> extends SinkFunction<IN> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void invoke(IN tuple) {
+ System.out.println(tuple);
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
new file mode 100644
index 0000000..cc4fb96
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
@@ -0,0 +1,33 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.sink;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+
+public abstract class SinkFunction<IN extends Tuple> extends AbstractFunction implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ public abstract void invoke(IN tuple);
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormat.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormat.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormat.java
new file mode 100644
index 0000000..18853b3
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormat.java
@@ -0,0 +1,47 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.sink;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Abstract class for formatting the output of the writeAsText and writeAsCsv
+ * functions.
+ *
+ * @param <IN>
+ * Input tuple type
+ */
+public abstract class WriteFormat<IN extends Tuple> implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Writes the contents of tupleList to the file specified by path.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param tupleList
+ * is the list of tuples to be written
+ */
+ protected abstract void write(String path, ArrayList<IN> tupleList);
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsCsv.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsCsv.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsCsv.java
new file mode 100644
index 0000000..e10a9c8
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsCsv.java
@@ -0,0 +1,53 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.sink;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Writes tuples in csv format.
+ *
+ * @param <IN>
+ * Input tuple type
+ */
+public class WriteFormatAsCsv<IN extends Tuple> extends WriteFormat<IN> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ protected void write(String path, ArrayList<IN> tupleList) {
+ try {
+ PrintWriter outStream = new PrintWriter(new BufferedWriter(new FileWriter(path, true)));
+ for (IN tupleToWrite : tupleList) {
+ outStream.println(tupleToWrite.toString().substring(1,
+ tupleToWrite.toString().length() - 1));
+ }
+ outStream.close();
+ } catch (IOException e) {
+ throw new RuntimeException("Exception occured while writing file " + path, e);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsText.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsText.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsText.java
new file mode 100644
index 0000000..2d591ae
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsText.java
@@ -0,0 +1,51 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.sink;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Writes tuples in text format.
+ *
+ * @param <IN>
+ * Input tuple type
+ */
+public class WriteFormatAsText<IN extends Tuple> extends WriteFormat<IN> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void write(String path, ArrayList<IN> tupleList) {
+ try {
+ PrintWriter outStream = new PrintWriter(new BufferedWriter(new FileWriter(path, true)));
+ for (IN tupleToWrite : tupleList) {
+ outStream.println(tupleToWrite);
+ }
+ outStream.close();
+ } catch (IOException e) {
+ throw new RuntimeException("Exception occured while writing file " + path, e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
new file mode 100644
index 0000000..d473190
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
@@ -0,0 +1,101 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.sink;
+
+import java.io.FileNotFoundException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Simple implementation of the SinkFunction writing tuples as simple text to
+ * the file specified by path. Tuples are collected to a list and written to the
+ * file periodically. The file specified by path is created if it does not
+ * exist, cleared if it exists before the writing.
+ *
+ * @param <IN>
+ * Input tuple type
+ */
+public abstract class WriteSinkFunction<IN extends Tuple> extends SinkFunction<IN> {
+ private static final long serialVersionUID = 1L;
+
+ protected final String path;
+ protected ArrayList<IN> tupleList = new ArrayList<IN>();
+ protected final IN endTuple;
+ protected WriteFormat<IN> format;
+
+ public WriteSinkFunction(String path, WriteFormat<IN> format, IN endTuple) {
+ this.path = path;
+ this.format = format;
+ this.endTuple = endTuple;
+ cleanFile(path);
+ }
+
+ /**
+ * Creates target file if it does not exist, cleans it if it exists.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ */
+ protected void cleanFile(String path) {
+ try {
+ PrintWriter writer;
+ writer = new PrintWriter(path);
+ writer.print("");
+ writer.close();
+ } catch (FileNotFoundException e) {
+ throw new RuntimeException("File not found " + path, e);
+ }
+ }
+
+ /**
+ * Condition for writing the contents of tupleList and clearing it.
+ *
+ * @return value of the updating condition
+ */
+ protected abstract boolean updateCondition();
+
+ /**
+ * Statements to be executed after writing a batch goes here.
+ */
+ protected abstract void resetParameters();
+
+ /**
+ * Implementation of the invoke method of the SinkFunction class. Collects
+ * the incoming tuples in tupleList and appends the list to the end of the
+ * target file if updateCondition() is true or the current tuple is the
+ * endTuple.
+ */
+ @Override
+ public void invoke(IN tuple) {
+ if (!tuple.equals(endTuple)) {
+ tupleList.add(tuple);
+ if (updateCondition()) {
+ format.write(path, tupleList);
+ resetParameters();
+ }
+ } else {
+ format.write(path, tupleList);
+ resetParameters();
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByBatches.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByBatches.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByBatches.java
new file mode 100644
index 0000000..3797d13
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByBatches.java
@@ -0,0 +1,52 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.sink;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Implementation of WriteSinkFunction. Writes tuples to file in equally sized
+ * batches.
+ *
+ * @param <IN>
+ * Input tuple type
+ */
+public class WriteSinkFunctionByBatches<IN extends Tuple> extends WriteSinkFunction<IN> {
+ private static final long serialVersionUID = 1L;
+
+ private final int batchSize;
+
+ public WriteSinkFunctionByBatches(String path, WriteFormat<IN> format, int batchSize,
+ IN endTuple) {
+ super(path, format, endTuple);
+ this.batchSize = batchSize;
+ }
+
+ @Override
+ protected boolean updateCondition() {
+ return tupleList.size() >= batchSize;
+ }
+
+ @Override
+ protected void resetParameters() {
+ tupleList.clear();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByMillis.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByMillis.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByMillis.java
new file mode 100644
index 0000000..cb77e6d
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByMillis.java
@@ -0,0 +1,54 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.sink;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Implementation of WriteSinkFunction. Writes tuples to file in every millis
+ * milliseconds.
+ *
+ * @param <IN>
+ * Input tuple type
+ */
+public class WriteSinkFunctionByMillis<IN extends Tuple> extends WriteSinkFunction<IN> {
+ private static final long serialVersionUID = 1L;
+
+ private final long millis;
+ private long lastTime;
+
+ public WriteSinkFunctionByMillis(String path, WriteFormat<IN> format, long millis, IN endTuple) {
+ super(path, format, endTuple);
+ this.millis = millis;
+ lastTime = System.currentTimeMillis();
+ }
+
+ @Override
+ protected boolean updateCondition() {
+ return System.currentTimeMillis() - lastTime >= millis;
+ }
+
+ @Override
+ protected void resetParameters() {
+ tupleList.clear();
+ lastTime = System.currentTimeMillis();
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
new file mode 100644
index 0000000..f016cbc
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
@@ -0,0 +1,53 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.source;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.IOException;
+
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.util.Collector;
+
+public class FileSourceFunction extends SourceFunction<Tuple1<String>> {
+ private static final long serialVersionUID = 1L;
+
+ private final String path;
+ private Tuple1<String> outTuple = new Tuple1<String>();
+
+ public FileSourceFunction(String path) {
+ this.path = path;
+ }
+
+ @Override
+ public void invoke(Collector<Tuple1<String>> collector) throws IOException {
+ BufferedReader br = new BufferedReader(new FileReader(path));
+ String line = br.readLine();
+ while (line != null) {
+ if (line != "") {
+ outTuple.f0 = line;
+ collector.collect(outTuple);
+ }
+ line = br.readLine();
+ }
+ br.close();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
new file mode 100644
index 0000000..c3ccedf
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
@@ -0,0 +1,54 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.source;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.IOException;
+
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.util.Collector;
+
+public class FileStreamFunction extends SourceFunction<Tuple1<String>>{
+ private static final long serialVersionUID = 1L;
+
+ private final String path;
+ private Tuple1<String> outTuple = new Tuple1<String>();
+
+ public FileStreamFunction(String path) {
+ this.path = path;
+ }
+
+ @Override
+ public void invoke(Collector<Tuple1<String>> collector) throws IOException {
+ while(true){
+ BufferedReader br = new BufferedReader(new FileReader(path));
+ String line = br.readLine();
+ while (line != null) {
+ if (line != "") {
+ outTuple.f0 = line;
+ collector.collect(outTuple);
+ }
+ line = br.readLine();
+ }
+ br.close();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
new file mode 100755
index 0000000..dfe29d2
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
@@ -0,0 +1,50 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.source;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.util.Collector;
+
+public class FromElementsFunction<T> extends SourceFunction<Tuple1<T>> {
+ private static final long serialVersionUID = 1L;
+
+ Iterable<T> iterable;
+ Tuple1<T> outTuple = new Tuple1<T>();
+
+ public FromElementsFunction(T... elements) {
+ this.iterable = Arrays.asList(elements);
+ }
+
+ public FromElementsFunction(Collection<T> elements) {
+ this.iterable = elements;
+ }
+
+ @Override
+ public void invoke(Collector<Tuple1<T>> collector) throws Exception {
+ for (T element : iterable) {
+ outTuple.f0 = element;
+ collector.collect(outTuple);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
new file mode 100755
index 0000000..706295e
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
@@ -0,0 +1,50 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.source;
+
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.util.Collector;
+
+/**
+ * Source Function used to generate the number sequence
+ *
+ */
+public class GenSequenceFunction extends SourceFunction<Tuple1<Long>> {
+
+ private static final long serialVersionUID = 1L;
+
+ long from;
+ long to;
+ Tuple1<Long> outTuple = new Tuple1<Long>();
+
+ public GenSequenceFunction(long from, long to) {
+ this.from = from;
+ this.to = to;
+ }
+
+ @Override
+ public void invoke(Collector<Tuple1<Long>> collector) throws Exception {
+ for (long i = from; i <= to; i++) {
+ outTuple.f0 = i;
+ collector.collect(outTuple);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
new file mode 100755
index 0000000..70553bf
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
@@ -0,0 +1,30 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.function.source;
+
+import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+public abstract class SourceFunction<OUT extends Tuple> extends UserSourceInvokable<OUT> {
+
+ private static final long serialVersionUID = 1L;
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index 2ba9da5..cb16307 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -19,9 +19,8 @@
package org.apache.flink.streaming.api.invokable;
-import org.apache.flink.streaming.api.function.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index d1c4e9b..731589b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -46,7 +46,7 @@ import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.collector.DirectedStreamCollector;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.collector.StreamCollector;
-import org.apache.flink.streaming.api.function.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 4e6e223..8ba58c5 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -26,7 +26,7 @@ import java.util.List;
import org.apache.flink.api.java.functions.FlatMapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.function.SinkFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.LogUtils;
import org.apache.flink.util.Collector;
import org.apache.log4j.Level;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
index c5cdd98..e296733 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsCsvTest.java
@@ -1,3 +1,22 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
package org.apache.flink.streaming.api;
import static org.junit.Assert.assertTrue;
@@ -10,7 +29,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.function.SourceFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/60f632a3/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
index 52a5cc7..fc0efcf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WriteAsTextTest.java
@@ -1,4 +1,4 @@
-/***********************************************************************************************************************
+/**
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*
- **********************************************************************************************************************/
+ */
package org.apache.flink.streaming.api;
@@ -29,7 +29,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.function.SourceFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
import org.junit.Test;
[20/51] [abbrv] git commit: [streaming] Added twitter-hbc dependency
to connectors and upgraded slf4j accordingly
Posted by se...@apache.org.
[streaming] Added twitter-hbc dependency to connectors and upgraded slf4j accordingly
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/7ca778b8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/7ca778b8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/7ca778b8
Branch: refs/heads/master
Commit: 7ca778b8ace79462a8907f7f24fce17851b95c0b
Parents: d282eef
Author: Márton Balassi <mb...@ilab.sztaki.hu>
Authored: Tue Jul 29 12:35:20 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:21:13 2014 +0200
----------------------------------------------------------------------
.../flink-streaming/flink-streaming-connectors/pom.xml | 10 ++++++++--
pom.xml | 6 +++---
2 files changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7ca778b8/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
index 6e9852d..ac15d55 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
@@ -49,13 +49,19 @@ under the License.
</dependency>
- <dependency>
+ <dependency>
<groupId>com.rabbitmq</groupId>
<artifactId>amqp-client</artifactId>
<version>3.3.1</version>
</dependency>
- </dependencies>
+ <dependency>
+ <groupId>com.twitter</groupId>
+ <artifactId>hbc-core</artifactId>
+ <version>2.2.0</version>
+ </dependency>
+
+ </dependencies>
<build>
<plugins>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7ca778b8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2baaeed..0394053 100644
--- a/pom.xml
+++ b/pom.xml
@@ -379,12 +379,12 @@ under the License.
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
- <version>1.4.3</version>
+ <version>1.6.6</version>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
- <version>1.4.3</version>
+ <version>1.6.6</version>
</dependency>
<dependency>
<groupId>org.javassist</groupId>
@@ -448,7 +448,7 @@ under the License.
<excludeSubProjects>false</excludeSubProjects>
<numUnapprovedLicenses>0</numUnapprovedLicenses>
<licenses>
- <!-- Enforce this license:
+ <!-- Enforce this license:
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
[45/51] [abbrv] git commit: [streaming] Iterative DataStreams updated
with buffer timout and max wait time
Posted by se...@apache.org.
[streaming] Iterative DataStreams updated with buffer timout and max wait time
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/1b31f4d1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/1b31f4d1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/1b31f4d1
Branch: refs/heads/master
Commit: 1b31f4d19df1fabc7aaee5837f4d2c3439819495
Parents: 9be9814
Author: gyfora <gy...@gmail.com>
Authored: Wed Aug 6 15:41:07 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:43 2014 +0200
----------------------------------------------------------------------
.../flink/streaming/api/JobGraphBuilder.java | 2 +-
.../flink/streaming/api/StreamConfig.java | 10 ++++++
.../streaming/api/datastream/DataStream.java | 17 ++++++----
.../api/datastream/DataStreamSink.java | 4 +--
.../api/datastream/IterativeDataStream.java | 34 +++++++++++++++-----
.../datastream/SingleOutputStreamOperator.java | 1 +
.../environment/StreamExecutionEnvironment.java | 20 ++++++++++--
.../AbstractStreamComponent.java | 6 ++--
.../streamcomponent/StreamIterationSink.java | 20 +++++++++---
.../streamcomponent/StreamIterationSource.java | 15 +++++++--
.../apache/flink/streaming/api/IterateTest.java | 13 +++-----
11 files changed, 104 insertions(+), 38 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index 7a10246..9280661 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -643,4 +643,4 @@ public class JobGraphBuilder {
return jobGraph;
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index 6fe9878..da1189e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -60,6 +60,7 @@ public class StreamConfig {
// STRINGS
private static final String MUTABILITY = "isMutable";
+ private static final String ITERATON_WAIT = "iterationWait";
private Configuration config;
@@ -196,6 +197,14 @@ public class StreamConfig {
return config.getString(ITERATION_ID, "iteration-0");
}
+ public void setIterationWaitTime(long time) {
+ config.setLong(ITERATON_WAIT, time);
+ }
+
+ public long getIterationWaitTime() {
+ return config.getLong(ITERATON_WAIT, 0);
+ }
+
public void setNumberOfOutputChannels(int outputIndex, Integer numberOfOutputChannels) {
config.setInteger(NUMBER_OF_OUTPUT_CHANNELS + outputIndex, numberOfOutputChannels);
}
@@ -267,4 +276,5 @@ public class StreamConfig {
ClassNotFoundException {
return (T) SerializationUtils.deserialize(serializedObject);
}
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index d15eaa5..7cab2df 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -169,7 +169,7 @@ public abstract class DataStream<OUT> {
}
/**
- * Creates a new {@link CoDataStream} bye connecting {@link DataStream}
+ * Creates a new {@link CoDataStream} by connecting {@link DataStream}
* outputs of different type with each other. The DataStreams connected
* using this operators can be used with CoFunctions.
*
@@ -676,9 +676,13 @@ public abstract class DataStream<OUT> {
* To direct tuples to the iteration head or the output specifically one can
* use the {@code split(OutputSelector)} on the iteration tail while
* referencing the iteration head as 'iterate'.
- *
+ * <p>
* The iteration edge will be partitioned the same way as the first input of
* the iteration head.
+ * <p>
+ * By default a DataStream with iteration will never terminate, but the user
+ * can use the {@link IterativeDataStream#setMaxWaitTime} call to set a max
+ * waiting time for the iteration.
*
* @return The iterative data stream created.
*/
@@ -686,12 +690,12 @@ public abstract class DataStream<OUT> {
return new IterativeDataStream<OUT>(this);
}
- protected <R> DataStream<OUT> addIterationSource(String iterationID) {
+ protected <R> DataStream<OUT> addIterationSource(String iterationID, long waitTime) {
DataStream<R> returnStream = new DataStreamSource<R>(environment, "iterationSource");
jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
- degreeOfParallelism);
+ degreeOfParallelism, waitTime);
return this.copy();
}
@@ -730,8 +734,9 @@ public abstract class DataStream<OUT> {
connectGraph(inputStream, returnStream.getId(), 0);
if (inputStream instanceof IterativeDataStream) {
- returnStream.addIterationSource(((IterativeDataStream<OUT>) inputStream).iterationID
- .toString());
+ IterativeDataStream<OUT> iterativeStream = (IterativeDataStream<OUT>) inputStream;
+ returnStream.addIterationSource(iterativeStream.iterationID.toString(),
+ iterativeStream.waitTime);
}
return returnStream;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
index ee6502f..1cc6ff2 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
@@ -27,7 +27,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
* @param <IN>
* The type of the DataStream closed by the sink.
*/
-public class DataStreamSink<IN> extends DataStream<IN> {
+public class DataStreamSink<IN> extends SingleOutputStreamOperator<IN, DataStreamSink<IN>> {
protected DataStreamSink(StreamExecutionEnvironment environment, String operatorType) {
super(environment, operatorType);
@@ -38,7 +38,7 @@ public class DataStreamSink<IN> extends DataStream<IN> {
}
@Override
- protected DataStream<IN> copy() {
+ protected DataStreamSink<IN> copy() {
throw new RuntimeException("Data stream sinks cannot be copied");
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
index bdadee4..f1cb13c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
@@ -31,20 +31,25 @@ import org.apache.flink.streaming.partitioner.ForwardPartitioner;
* @param <IN>
* Type of the DataStream
*/
-public class IterativeDataStream<IN> extends SingleOutputStreamOperator<IN, IterativeDataStream<IN>> {
+public class IterativeDataStream<IN> extends
+ SingleOutputStreamOperator<IN, IterativeDataStream<IN>> {
static Integer iterationCount = 0;
protected Integer iterationID;
+ protected long waitTime;
protected IterativeDataStream(DataStream<IN> dataStream) {
super(dataStream);
+ setBufferTimeout(dataStream.environment.getBufferTimeout());
iterationID = iterationCount;
iterationCount++;
+ waitTime = 0;
}
- protected IterativeDataStream(DataStream<IN> dataStream, Integer iterationID) {
+ protected IterativeDataStream(DataStream<IN> dataStream, Integer iterationID, long waitTime) {
super(dataStream);
this.iterationID = iterationID;
+ this.waitTime = waitTime;
}
/**
@@ -80,18 +85,17 @@ public class IterativeDataStream<IN> extends SingleOutputStreamOperator<IN, Iter
DataStream<R> returnStream = new DataStreamSink<R>(environment, "iterationSink");
jobGraphBuilder.addIterationSink(returnStream.getId(), iterationTail.getId(),
- iterationID.toString(), iterationTail.getParallelism());
+ iterationID.toString(), iterationTail.getParallelism(), waitTime);
- jobGraphBuilder.setIterationSourceParallelism(iterationID.toString(),
- iterationTail.getParallelism());
+ jobGraphBuilder.setIterationSourceSettings(iterationID.toString(), iterationTail.getId());
List<String> name = Arrays.asList(new String[] { iterationName });
if (iterationTail instanceof ConnectedDataStream) {
for (DataStream<IN> stream : ((ConnectedDataStream<IN>) iterationTail).connectedStreams) {
String inputID = stream.getId();
- jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<IN>(),
- 0, name);
+ jobGraphBuilder.setEdge(inputID, returnStream.getId(),
+ new ForwardPartitioner<IN>(), 0, name);
}
} else {
@@ -102,8 +106,22 @@ public class IterativeDataStream<IN> extends SingleOutputStreamOperator<IN, Iter
return iterationTail;
}
+ /**
+ * Sets the max waiting time for the next record before shutting down the
+ * stream. If not set, then the user needs to manually kill the process to
+ * stop.
+ *
+ * @param waitTimeMillis
+ * Max waiting time in milliseconds
+ * @return The modified DataStream.
+ */
+ public IterativeDataStream<IN> setMaxWaitTime(long waitTimeMillis) {
+ this.waitTime = waitTimeMillis;
+ return this;
+ }
+
@Override
protected IterativeDataStream<IN> copy() {
- return new IterativeDataStream<IN>(this, iterationID);
+ return new IterativeDataStream<IN>(this, iterationID, waitTime);
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index f798563..6d660b1 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -38,6 +38,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, String operatorType) {
super(environment, operatorType);
+ setBufferTimeout(environment.getBufferTimeout());
}
protected SingleOutputStreamOperator(DataStream<OUT> dataStream) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 3773d8e..c357424 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -60,6 +60,8 @@ public abstract class StreamExecutionEnvironment {
private int executionParallelism = -1;
+ private long buffertimeout = 0;;
+
protected JobGraphBuilder jobGraphBuilder;
// --------------------------------------------------------------------------------------------
@@ -110,6 +112,21 @@ public abstract class StreamExecutionEnvironment {
}
/**
+ * Sets the maximum time frequency (ms) for the flushing of the output
+ * buffers. By default the output buffers flush only when they are full.
+ *
+ * @param timeoutMillis
+ * The maximum time between two output flushes.
+ */
+ public void setBufferTimeout(long timeoutMillis) {
+ this.buffertimeout = timeoutMillis;
+ }
+
+ public long getBufferTimeout() {
+ return this.buffertimeout;
+ }
+
+ /**
* Sets the number of hardware contexts (CPU cores / threads) used when
* executed in {@link LocalStreamEnvironment}.
*
@@ -178,8 +195,7 @@ public abstract class StreamExecutionEnvironment {
* @return The DataStream representing the elements.
*/
public <OUT extends Serializable> DataStreamSource<OUT> fromElements(OUT... data) {
- DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(
- this, "elements");
+ DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements");
try {
SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index 76dee5d..e2e8816 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -64,6 +64,7 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
protected boolean isMutable;
protected Object function;
protected String functionName;
+ protected long bufferTimeout;
protected static int newComponent() {
numComponents++;
@@ -115,7 +116,8 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
setCollector();
int numberOfOutputs = configuration.getNumberOfOutputs();
-
+ bufferTimeout= configuration.getBufferTimeout();
+
for (int i = 0; i < numberOfOutputs; i++) {
setPartitioner(i, outputs);
}
@@ -130,8 +132,6 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output;
- long bufferTimeout = configuration.getBufferTimeout();
-
if (bufferTimeout > 0) {
output = new StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>>(this,
outputPartitioner, bufferTimeout);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
index 436ebbf..5586887 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
@@ -38,6 +38,8 @@ public class StreamIterationSink<IN extends Tuple> extends
private String iterationId;
@SuppressWarnings("rawtypes")
private BlockingQueue<StreamRecord> dataChannel;
+ private long iterationWaitTime;
+ private boolean shouldWait;
public StreamIterationSink() {
}
@@ -51,6 +53,8 @@ public class StreamIterationSink<IN extends Tuple> extends
inputIter = createInputIterator(inputs, inputSerializer);
iterationId = configuration.getIterationId();
+ iterationWaitTime = configuration.getIterationWaitTime();
+ shouldWait = iterationWaitTime > 0;
dataChannel = BlockingQueueBroker.instance().get(iterationId);
} catch (Exception e) {
@@ -73,22 +77,30 @@ public class StreamIterationSink<IN extends Tuple> extends
}
protected void forwardRecords() throws Exception {
- StreamRecord<IN> reuse = inputSerializer.createInstance().setId(0);
+ StreamRecord<IN> reuse = inputSerializer.createInstance();
while ((reuse = inputIter.next(reuse)) != null) {
- pushToQueue(reuse);
+ if (!pushToQueue(reuse)) {
+ break;
+ }
// TODO: Fix object reuse for iteration
reuse = inputSerializer.createInstance();
}
}
- private void pushToQueue(StreamRecord<IN> record) {
+ private boolean pushToQueue(StreamRecord<IN> record) {
try {
- dataChannel.offer(record, 5, TimeUnit.MILLISECONDS);
+ if (shouldWait) {
+ return dataChannel.offer(record, iterationWaitTime, TimeUnit.MILLISECONDS);
+ } else {
+ dataChannel.put(record);
+ return true;
+ }
} catch (InterruptedException e) {
if (LOG.isErrorEnabled()) {
LOG.error(String.format("Pushing back record at iteration %s failed due to: %s",
iterationId, StringUtils.stringifyException(e)));
}
+ return false;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
index 67d5066..2514eb0 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
@@ -42,6 +42,8 @@ public class StreamIterationSource<OUT extends Tuple> extends
private String iterationId;
@SuppressWarnings("rawtypes")
private BlockingQueue<StreamRecord> dataChannel;
+ private long iterationWaitTime;
+ private boolean shouldWait;
@SuppressWarnings("rawtypes")
public StreamIterationSource() {
@@ -56,13 +58,15 @@ public class StreamIterationSource<OUT extends Tuple> extends
public void setInputsOutputs() {
try {
setConfigOutputs(outputs);
- setSinkSerializer();
} catch (StreamComponentException e) {
e.printStackTrace();
throw new StreamComponentException("Cannot register outputs", e);
}
iterationId = configuration.getIterationId();
+ iterationWaitTime = configuration.getIterationWaitTime();
+ shouldWait = iterationWaitTime > 0;
+
try {
BlockingQueueBroker.instance().handIn(iterationId, dataChannel);
} catch (Exception e) {
@@ -71,6 +75,7 @@ public class StreamIterationSource<OUT extends Tuple> extends
}
+ @SuppressWarnings("unchecked")
@Override
public void invoke() throws Exception {
if (LOG.isDebugEnabled()) {
@@ -80,10 +85,14 @@ public class StreamIterationSource<OUT extends Tuple> extends
for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
output.initializeSerializers();
}
+ StreamRecord<OUT> nextRecord;
while (true) {
- @SuppressWarnings("unchecked")
- StreamRecord<OUT> nextRecord = dataChannel.poll(3, TimeUnit.SECONDS);
+ if (shouldWait) {
+ nextRecord = dataChannel.poll(iterationWaitTime, TimeUnit.MILLISECONDS);
+ } else {
+ nextRecord = dataChannel.take();
+ }
if (nextRecord == null) {
break;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1b31f4d1/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 5872da9..9498b8e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -21,9 +21,6 @@ package org.apache.flink.streaming.api;
import static org.junit.Assert.assertTrue;
-import java.util.ArrayList;
-import java.util.List;
-
import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.IterativeDataStream;
@@ -84,13 +81,11 @@ public class IterateTest {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
- List<Boolean> bl = new ArrayList<Boolean>();
- for (int i = 0; i < 100000; i++) {
- bl.add(false);
- }
- DataStream<Boolean> source = env.fromCollection(bl);
+ env.setBufferTimeout(10);
+
+ DataStream<Boolean> source = env.fromElements(false, false, false);
- IterativeDataStream<Boolean> iteration = source.iterate();
+ IterativeDataStream<Boolean> iteration = source.iterate().setMaxWaitTime(3000);
DataStream<Boolean> increment = iteration.flatMap(new IterationHead()).flatMap(
new IterationTail());
[49/51] [abbrv] git commit: [streaming] GroupReduce operator added +
StreamCollector bugfix
Posted by se...@apache.org.
[streaming] GroupReduce operator added + StreamCollector bugfix
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/309727ef
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/309727ef
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/309727ef
Branch: refs/heads/master
Commit: 309727ef94cb0bdca7e82ea31f020abf2c7075e4
Parents: 74d3742
Author: gyfora <gy...@gmail.com>
Authored: Wed Aug 6 22:36:53 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:43 2014 +0200
----------------------------------------------------------------------
.../connectors/twitter/TwitterLocal.java | 34 +++++++---
.../api/collector/StreamCollector.java | 12 ++--
.../streaming/api/datastream/DataStream.java | 46 ++++++++++---
.../datastream/SingleOutputStreamOperator.java | 4 +-
.../streaming/api/invokable/SinkInvokable.java | 16 +----
.../api/invokable/SourceInvokable.java | 18 +----
.../api/invokable/StreamComponentInvokable.java | 15 ++++-
.../api/invokable/StreamRecordInvokable.java | 5 ++
.../api/invokable/UserTaskInvokable.java | 6 ++
.../operator/BatchReduceInvokable.java | 1 +
.../api/invokable/operator/FilterInvokable.java | 16 +----
.../invokable/operator/FlatMapInvokable.java | 16 +----
.../operator/GroupReduceInvokable.java | 71 ++++++++++++++++++++
.../api/invokable/operator/MapInvokable.java | 17 +----
.../operator/StreamReduceInvokable.java | 18 ++---
.../operator/WindowReduceInvokable.java | 1 +
.../api/invokable/operator/co/CoInvokable.java | 5 ++
.../invokable/operator/co/CoMapInvokable.java | 1 +
.../AbstractStreamComponent.java | 5 +-
.../api/collector/StreamCollectorTest.java | 6 +-
.../examples/wordcount/WordCountCounter.java | 32 ++-------
.../examples/wordcount/WordCountLocal.java | 5 +-
.../examples/wordcount/WordCountSplitter.java | 7 +-
23 files changed, 203 insertions(+), 154 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
index 878a7ad..668647d 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
@@ -19,6 +19,9 @@
package org.apache.flink.streaming.connectors.twitter;
+import java.io.Serializable;
+
+import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -28,19 +31,19 @@ import org.apache.flink.streaming.examples.wordcount.WordCountCounter;
import org.apache.flink.util.Collector;
/**
- * This program demonstrate the use of TwitterSource.
- * Its aim is to count the frequency of the languages of tweets
+ * This program demonstrate the use of TwitterSource. Its aim is to count the
+ * frequency of the languages of tweets
*/
-public class TwitterLocal {
+public class TwitterLocal implements Serializable {
+ private static final long serialVersionUID = 1L;
private static final int PARALLELISM = 1;
private static final int SOURCE_PARALLELISM = 1;
/**
- * FlatMapFunction to determine the language of tweets if possible
+ * FlatMapFunction to determine the language of tweets if possible
*/
- public static class SelectLanguageFlatMap extends
- JSONParseFlatMap<String, String> {
+ public static class SelectLanguageFlatMap extends JSONParseFlatMap<String, String> {
private static final long serialVersionUID = 1L;
@@ -54,7 +57,9 @@ public class TwitterLocal {
}
/**
- * Change the null String to space character. Useful when null is undesirable.
+ * Change the null String to space character. Useful when null is
+ * undesirable.
+ *
* @param in
* @return
*/
@@ -83,11 +88,18 @@ public class TwitterLocal {
DataStream<String> streamSource = env.addSource(new TwitterSource(path, 100),
SOURCE_PARALLELISM);
-
DataStream<Tuple2<String, Integer>> dataStream = streamSource
- .flatMap(new SelectLanguageFlatMap())
- .partitionBy(0)
- .map(new WordCountCounter());
+ .flatMap(new SelectLanguageFlatMap()).partitionBy(0)
+ .map(new MapFunction<String, Tuple2<String, Integer>>() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public Tuple2<String, Integer> map(String value) throws Exception {
+
+ return new Tuple2<String, Integer>(value, 1);
+ }
+ }).groupReduce(new WordCountCounter(), 0);
dataStream.addSink(new SinkFunction<Tuple2<String, Integer>>() {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
index fe21c29..e76cf94 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
@@ -60,9 +60,12 @@ public class StreamCollector<OUT> implements Collector<OUT> {
*/
public StreamCollector(int channelID,
SerializationDelegate<StreamRecord<OUT>> serializationDelegate) {
-
this.serializationDelegate = serializationDelegate;
- this.streamRecord = new StreamRecord<OUT>();
+ if (serializationDelegate != null) {
+ this.streamRecord = serializationDelegate.getInstance();
+ } else {
+ this.streamRecord = new StreamRecord<OUT>();
+ }
this.channelID = channelID;
this.outputs = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
this.outputMap = new HashMap<String, List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>>();
@@ -82,8 +85,9 @@ public class StreamCollector<OUT> implements Collector<OUT> {
for (String outputName : outputNames) {
if (outputName != null) {
if (!outputMap.containsKey(outputName)) {
- outputMap.put(outputName,
- new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>());
+ outputMap
+ .put(outputName,
+ new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>());
outputMap.get(outputName).add(output);
} else {
if (!outputMap.get(outputName).contains(output)) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 86b3322..bec55e0 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -30,10 +30,12 @@ import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.GroupReduceFunction;
import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.java.functions.RichFilterFunction;
import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.api.java.functions.RichGroupReduceFunction;
import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.java.functions.RichReduceFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.JobGraphBuilder;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -48,6 +50,7 @@ import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
+import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
import org.apache.flink.streaming.api.invokable.operator.WindowReduceInvokable;
import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
@@ -185,16 +188,16 @@ public abstract class DataStream<OUT> {
* Sets the partitioning of the {@link DataStream} so that the output tuples
* are partitioned by their hashcode and are sent to only one component.
*
- * @param keyposition
+ * @param keyPosition
* The field used to compute the hashcode.
* @return The DataStream with field partitioning set.
*/
- public DataStream<OUT> partitionBy(int keyposition) {
- if (keyposition < 0) {
+ public DataStream<OUT> partitionBy(int keyPosition) {
+ if (keyPosition < 0) {
throw new IllegalArgumentException("The position of the field must be non-negative");
}
- return setConnectionType(new FieldsPartitioner<OUT>(keyposition));
+ return setConnectionType(new FieldsPartitioner<OUT>(keyPosition));
}
/**
@@ -280,6 +283,29 @@ public abstract class DataStream<OUT> {
}
/**
+ * Applies a group and a reduce transformation on the DataStream grouped on
+ * the given key position. The {@link ReduceFunction} will receive input
+ * values based on the key value. Only input values with the same key will
+ * go to the same reducer.The user can also extend
+ * {@link RichReduceFunction} to gain access to other features provided
+ * by the {@link RichFuntion} interface.
+ *
+ * @param reducer
+ * The {@link ReduceFunction} that will be called for every
+ * element of the input values with the same key.
+ * @param keyPosition
+ * The key position in the input values on which the grouping is
+ * made.
+ * @return The transformed DataStream.
+ */
+ public SingleOutputStreamOperator<OUT, ?> groupReduce(ReduceFunction<OUT> reducer,
+ int keyPosition) {
+ return addFunction("groupReduce", reducer,
+ new FunctionTypeWrapper<OUT, Tuple, OUT>(reducer, ReduceFunction.class, 0, -1, 0),
+ new GroupReduceInvokable<OUT>(reducer, keyPosition)).partitionBy(keyPosition);
+ }
+
+ /**
* Applies a reduce transformation on preset chunks of the DataStream. The
* transformation calls a {@link GroupReduceFunction} for each tuple batch
* of the predefined size. Each GroupReduceFunction call can return any
@@ -505,8 +531,8 @@ public abstract class DataStream<OUT> {
*/
private DataStreamSink<OUT> writeAsText(DataStream<OUT> inputStream, String path,
WriteFormatAsText<OUT> format, int batchSize, OUT endTuple) {
- DataStreamSink<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<OUT>(
- path, format, batchSize, endTuple), null);
+ DataStreamSink<OUT> returnStream = addSink(inputStream,
+ new WriteSinkFunctionByBatches<OUT>(path, format, batchSize, endTuple), null);
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
jobGraphBuilder.setMutability(returnStream.getId(), false);
return returnStream;
@@ -657,8 +683,8 @@ public abstract class DataStream<OUT> {
*/
private DataStreamSink<OUT> writeAsCsv(DataStream<OUT> inputStream, String path,
WriteFormatAsCsv<OUT> format, int batchSize, OUT endTuple) {
- DataStreamSink<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<OUT>(
- path, format, batchSize, endTuple), null);
+ DataStreamSink<OUT> returnStream = addSink(inputStream,
+ new WriteSinkFunctionByBatches<OUT>(path, format, batchSize, endTuple), null);
jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
jobGraphBuilder.setMutability(returnStream.getId(), false);
return returnStream;
@@ -801,8 +827,8 @@ public abstract class DataStream<OUT> {
sinkFunction, SinkFunction.class, 0, -1, 0));
}
- private DataStreamSink<OUT> addSink(DataStream<OUT> inputStream, SinkFunction<OUT> sinkFunction,
- TypeSerializerWrapper<OUT, Tuple, OUT> typeWrapper) {
+ private DataStreamSink<OUT> addSink(DataStream<OUT> inputStream,
+ SinkFunction<OUT> sinkFunction, TypeSerializerWrapper<OUT, Tuple, OUT> typeWrapper) {
DataStreamSink<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink");
try {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index 6d660b1..b86fac0 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -77,7 +77,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
* The mutability of the operator.
* @return The operator with mutability set.
*/
- public DataStream<OUT> setMutability(boolean isMutable) {
+ public SingleOutputStreamOperator<OUT, O> setMutability(boolean isMutable) {
jobGraphBuilder.setMutability(id, isMutable);
return this;
}
@@ -90,7 +90,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
* The maximum time between two output flushes.
* @return The operator with buffer timeout set.
*/
- public DataStream<OUT> setBufferTimeout(long timeoutMillis) {
+ public SingleOutputStreamOperator<OUT, O> setBufferTimeout(long timeoutMillis) {
jobGraphBuilder.setBufferTimeout(id, timeoutMillis);
return this;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index 50bdd42..887df8b 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -19,8 +19,6 @@
package org.apache.flink.streaming.api.invokable;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
public class SinkInvokable<IN> extends StreamRecordInvokable<IN, IN> {
@@ -29,6 +27,7 @@ public class SinkInvokable<IN> extends StreamRecordInvokable<IN, IN> {
private SinkFunction<IN> sinkFunction;
public SinkInvokable(SinkFunction<IN> sinkFunction) {
+ super(sinkFunction);
this.sinkFunction = sinkFunction;
}
@@ -47,17 +46,4 @@ public class SinkInvokable<IN> extends StreamRecordInvokable<IN, IN> {
}
}
- @Override
- public void open(Configuration parameters) throws Exception {
- if (sinkFunction instanceof RichFunction) {
- ((RichFunction) sinkFunction).open(parameters);
- }
- }
-
- @Override
- public void close() throws Exception {
- if (sinkFunction instanceof RichFunction) {
- ((RichFunction) sinkFunction).close();
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
index d7710ae..a4be1e8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
@@ -21,8 +21,6 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.function.source.SourceFunction;
public class SourceInvokable<OUT> extends StreamComponentInvokable<OUT> implements Serializable {
@@ -31,10 +29,9 @@ public class SourceInvokable<OUT> extends StreamComponentInvokable<OUT> implemen
private SourceFunction<OUT> sourceFunction;
- public SourceInvokable() {
- }
public SourceInvokable(SourceFunction<OUT> sourceFunction) {
+ super(sourceFunction);
this.sourceFunction = sourceFunction;
}
@@ -42,17 +39,4 @@ public class SourceInvokable<OUT> extends StreamComponentInvokable<OUT> implemen
sourceFunction.invoke(collector);
}
- @Override
- public void open(Configuration parameters) throws Exception {
- if (sourceFunction instanceof RichFunction) {
- ((RichFunction) sourceFunction).open(parameters);
- }
- }
-
- @Override
- public void close() throws Exception {
- if (sourceFunction instanceof RichFunction) {
- ((RichFunction) sourceFunction).close();
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
index 02ee5fd..ed718f1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
@@ -22,6 +22,8 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.RichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.Collector;
@@ -35,6 +37,11 @@ public abstract class StreamComponentInvokable<OUT> extends AbstractRichFunction
@SuppressWarnings("unused")
private int channelID;
protected Collector<OUT> collector;
+ protected Function userFunction;
+
+ public StreamComponentInvokable(Function userFunction) {
+ this.userFunction = userFunction;
+ }
public void setCollector(Collector<OUT> collector) {
this.collector = collector;
@@ -47,12 +54,16 @@ public abstract class StreamComponentInvokable<OUT> extends AbstractRichFunction
@Override
public void open(Configuration parameters) throws Exception {
- System.out.println("Open not implemented");
+ if (userFunction instanceof RichFunction) {
+ ((RichFunction) userFunction).open(parameters);
+ }
}
@Override
public void close() throws Exception {
- System.out.println("Close not implemented");
+ if (userFunction instanceof RichFunction) {
+ ((RichFunction) userFunction).close();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
index b1cdde1..27dc05a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.api.invokable;
import java.io.IOException;
+import org.apache.flink.api.common.functions.Function;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.util.Collector;
@@ -29,6 +30,10 @@ import org.apache.flink.util.MutableObjectIterator;
public abstract class StreamRecordInvokable<IN, OUT> extends
StreamComponentInvokable<OUT> {
+ public StreamRecordInvokable(Function userFunction) {
+ super(userFunction);
+ }
+
private static final long serialVersionUID = 1L;
protected MutableObjectIterator<StreamRecord<IN>> recordIterator;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java
index 0a88efd..8d1da6b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserTaskInvokable.java
@@ -21,8 +21,14 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
+import org.apache.flink.api.common.functions.Function;
+
public abstract class UserTaskInvokable<IN, OUT> extends
StreamRecordInvokable<IN, OUT> implements Serializable {
+ public UserTaskInvokable(Function userFunction) {
+ super(userFunction);
+ }
+
private static final long serialVersionUID = 1L;
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index c3c861b..4e0a7a5 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -29,6 +29,7 @@ public class BatchReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT
private int batchSize;
public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, int batchSize) {
+ super(reduceFunction);
this.reducer = reduceFunction;
this.batchSize = batchSize;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
index b64f08b..388920c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
@@ -20,8 +20,6 @@
package org.apache.flink.streaming.api.invokable.operator;
import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
@@ -31,6 +29,7 @@ public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
FilterFunction<IN> filterFunction;
public FilterInvokable(FilterFunction<IN> filterFunction) {
+ super(filterFunction);
this.filterFunction = filterFunction;
}
@@ -53,17 +52,4 @@ public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
}
}
- @Override
- public void open(Configuration parameters) throws Exception {
- if (filterFunction instanceof RichFunction) {
- ((RichFunction) filterFunction).open(parameters);
- }
- }
-
- @Override
- public void close() throws Exception {
- if (filterFunction instanceof RichFunction) {
- ((RichFunction) filterFunction).close();
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
index cc9fcc1..4cb4712 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
@@ -20,8 +20,6 @@
package org.apache.flink.streaming.api.invokable.operator;
import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
@@ -30,6 +28,7 @@ public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private FlatMapFunction<IN, OUT> flatMapper;
public FlatMapInvokable(FlatMapFunction<IN, OUT> flatMapper) {
+ super(flatMapper);
this.flatMapper = flatMapper;
}
@@ -48,17 +47,4 @@ public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
}
}
- @Override
- public void open(Configuration parameters) throws Exception {
- if (flatMapper instanceof RichFunction) {
- ((RichFunction) flatMapper).open(parameters);
- }
- }
-
- @Override
- public void close() throws Exception {
- if (flatMapper instanceof RichFunction) {
- ((RichFunction) flatMapper).close();
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
new file mode 100755
index 0000000..67af978
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
@@ -0,0 +1,71 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
+
+public class GroupReduceInvokable<IN> extends UserTaskInvokable<IN, IN> {
+ private static final long serialVersionUID = 1L;
+
+ private ReduceFunction<IN> reducer;
+ private int keyPosition;
+ private Map<Object, IN> values;
+
+ public GroupReduceInvokable(ReduceFunction<IN> reducer, int keyPosition) {
+ super(reducer);
+ this.reducer = reducer;
+ this.keyPosition = keyPosition;
+ values = new HashMap<Object, IN>();
+ }
+
+ @Override
+ protected void immutableInvoke() throws Exception {
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ reduce();
+ resetReuse();
+ }
+ }
+
+ @Override
+ protected void mutableInvoke() throws Exception {
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ reduce();
+ }
+ }
+
+ private void reduce() throws Exception {
+ Object key = reuse.getField(keyPosition);
+ IN currentValue = values.get(key);
+ IN nextValue = reuse.getObject();
+ if (currentValue != null) {
+ IN reduced = reducer.reduce(currentValue, nextValue);
+ values.put(key, reduced);
+ collector.collect(reduced);
+ } else {
+ values.put(key, nextValue);
+ collector.collect(nextValue);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
index 9dbb678..53e85e0 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
@@ -20,8 +20,6 @@
package org.apache.flink.streaming.api.invokable.operator;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
@@ -30,6 +28,7 @@ public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
private MapFunction<IN, OUT> mapper;
public MapInvokable(MapFunction<IN, OUT> mapper) {
+ super(mapper);
this.mapper = mapper;
}
@@ -47,18 +46,4 @@ public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
collector.collect(mapper.map(reuse.getObject()));
}
}
-
- @Override
- public void open(Configuration parameters) throws Exception {
- if (mapper instanceof RichFunction) {
- ((RichFunction) mapper).open(parameters);
- }
- }
-
- @Override
- public void close() throws Exception {
- if (mapper instanceof RichFunction) {
- ((RichFunction) mapper).close();
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
index 548a298..de0a18a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
@@ -21,12 +21,17 @@ package org.apache.flink.streaming.api.invokable.operator;
import java.util.Iterator;
+import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.RichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public abstract class StreamReduceInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
+
+ public StreamReduceInvokable(Function userFunction) {
+ super(userFunction);
+ }
+
private static final long serialVersionUID = 1L;
protected GroupReduceFunction<IN, OUT> reducer;
protected BatchIterator<IN> userIterator;
@@ -35,16 +40,7 @@ public abstract class StreamReduceInvokable<IN, OUT> extends UserTaskInvokable<I
@Override
public void open(Configuration parameters) throws Exception {
userIterable = new BatchIterable();
- if (reducer instanceof RichFunction) {
- ((RichFunction) reducer).open(parameters);
- }
- }
-
- @Override
- public void close() throws Exception {
- if (reducer instanceof RichFunction) {
- ((RichFunction) reducer).close();
- }
+ super.open(parameters);
}
protected class BatchIterable implements Iterable<IN> {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index 309656b..3405641 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -31,6 +31,7 @@ public class WindowReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OU
boolean window;
public WindowReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize) {
+ super(reduceFunction);
this.reducer = reduceFunction;
this.windowSize = windowSize;
this.window = true;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
index d854e89..4ac75a1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.api.invokable.operator.co;
+import org.apache.flink.api.common.functions.Function;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
@@ -28,6 +29,10 @@ import org.apache.flink.util.MutableObjectIterator;
public abstract class CoInvokable<IN1, IN2, OUT> extends
StreamComponentInvokable<OUT> {
+ public CoInvokable(Function userFunction) {
+ super(userFunction);
+ }
+
private static final long serialVersionUID = 1L;
protected Collector<OUT> collector;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
index cd51081..ab10b40 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
@@ -29,6 +29,7 @@ public class CoMapInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
private CoMapFunction<IN1, IN2, OUT> mapper;
public CoMapInvokable(CoMapFunction<IN1, IN2, OUT> mapper) {
+ super(mapper);
this.mapper = mapper;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index e2e8816..324b5ba 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -108,6 +108,7 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
outTypeInfo = typeWrapper.getOutputTypeInfo();
outSerializer = new StreamRecordSerializer<OUT>(outTypeInfo);
outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(outSerializer);
+ outSerializationDelegate.setInstance(outSerializer.createInstance());
}
protected void setConfigOutputs(
@@ -116,8 +117,8 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
setCollector();
int numberOfOutputs = configuration.getNumberOfOutputs();
- bufferTimeout= configuration.getBufferTimeout();
-
+ bufferTimeout = configuration.getBufferTimeout();
+
for (int i = 0; i < numberOfOutputs; i++) {
setPartitioner(i, outputs);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
index 72b09c9..99f05ac 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
@@ -35,9 +35,11 @@ public class StreamCollectorTest {
@Test
public void testCollect() {
MockRecordWriter recWriter = MockRecordWriterFactory.create();
+ SerializationDelegate<StreamRecord<Tuple1<Integer>>> sd = new SerializationDelegate<StreamRecord<Tuple1<Integer>>>(
+ null);
+ sd.setInstance(new StreamRecord<Tuple1<Integer>>().setObject(new Tuple1<Integer>()));
- StreamCollector<Tuple1<Integer>> collector = new StreamCollector<Tuple1<Integer>>(2,
- new SerializationDelegate<StreamRecord<Tuple1<Integer>>>(null));
+ StreamCollector<Tuple1<Integer>> collector = new StreamCollector<Tuple1<Integer>>(2, sd);
collector.addOutput(recWriter, new ArrayList<String>());
collector.collect(new Tuple1<Integer>(3));
collector.collect(new Tuple1<Integer>(4));
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
index f106afe..6ee824a 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
@@ -19,38 +19,16 @@
package org.apache.flink.streaming.examples.wordcount;
-import java.util.HashMap;
-import java.util.Map;
-
+import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.common.functions.MapFunction;
-public class WordCountCounter implements MapFunction<String, Tuple2<String, Integer>> {
+public class WordCountCounter implements ReduceFunction<Tuple2<String, Integer>> {
private static final long serialVersionUID = 1L;
- private Map<String, Integer> wordCounts = new HashMap<String, Integer>();
- private String word = "";
- private Integer count = 0;
-
- private Tuple2<String, Integer> outTuple = new Tuple2<String, Integer>();
-
- // Increments the counter of the occurrence of the input word
@Override
- public Tuple2<String, Integer> map(String inTuple) throws Exception {
- word = inTuple;
-
- if (wordCounts.containsKey(word)) {
- count = wordCounts.get(word) + 1;
- wordCounts.put(word, count);
- } else {
- count = 1;
- wordCounts.put(word, 1);
- }
-
- outTuple.f0 = word;
- outTuple.f1 = count;
-
- return outTuple;
+ public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1,
+ Tuple2<String, Integer> value2) throws Exception {
+ return new Tuple2<String, Integer>(value1.f0, value1.f1 + value2.f1);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
index fc31930..ba5a6e2 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
@@ -19,10 +19,10 @@
package org.apache.flink.streaming.examples.wordcount;
+import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.TestDataUtil;
-import org.apache.flink.api.java.tuple.Tuple2;
public class WordCountLocal {
@@ -35,7 +35,8 @@ public class WordCountLocal {
DataStream<Tuple2<String, Integer>> dataStream = env
.readTextFile("src/test/resources/testdata/hamlet.txt")
- .flatMap(new WordCountSplitter()).partitionBy(0).map(new WordCountCounter());
+ .flatMap(new WordCountSplitter())
+ .groupReduce(new WordCountCounter(), 0);
dataStream.print();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/309727ef/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
index eb0dfd3..b962d4d 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
@@ -20,17 +20,18 @@
package org.apache.flink.streaming.examples.wordcount;
import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.util.Collector;
-public class WordCountSplitter implements FlatMapFunction<String, String> {
+public class WordCountSplitter implements FlatMapFunction<String, Tuple2<String, Integer>> {
private static final long serialVersionUID = 1L;
// Splits the lines according on spaces
@Override
- public void flatMap(String inTuple, Collector<String> out) throws Exception {
+ public void flatMap(String inTuple, Collector<Tuple2<String, Integer>> out) throws Exception {
for (String word : inTuple.split(" ")) {
- out.collect(word);
+ out.collect(new Tuple2<String, Integer>(word, 1));
}
}
}
\ No newline at end of file
[14/51] [abbrv] git commit: [streaming] Distributed partitioner added
Posted by se...@apache.org.
[streaming] Distributed partitioner added
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/2c4e195c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/2c4e195c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/2c4e195c
Branch: refs/heads/master
Commit: 2c4e195c7e0e2e2b984ac3261c9272940b4d21dc
Parents: b92ce01
Author: Tofi007 <ba...@gmail.com>
Authored: Tue Jul 22 14:57:00 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:19:17 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 15 +++++
.../flink/streaming/api/JobGraphBuilder.java | 19 +++++++
.../api/StreamExecutionEnvironment.java | 5 +-
.../partitioner/DistributePartitioner.java | 51 +++++++++++++++++
.../partitioner/DistributePartitionerTest.java | 58 ++++++++++++++++++++
.../partitioner/ForwardPartitionerTest.java | 57 +++++++++++++++++++
6 files changed, 204 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2c4e195c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index 6ef2faf..23f8408 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -314,6 +314,21 @@ public class DataStream<T extends Tuple> {
}
return returnStream;
}
+
+ /**
+ * Sets the partitioning of the {@link DataStream} so that the output tuples
+ * are distributed evenly to the next component.
+ *
+ * @return The DataStream with shuffle partitioning set.
+ */
+ public DataStream<T> distribute() {
+ DataStream<T> returnStream = new DataStream<T>(this);
+
+ for (int i = 0; i < returnStream.ctypes.size(); i++) {
+ returnStream.ctypes.set(i, ConnectionType.DISTRIBUTE);
+ }
+ return returnStream;
+ }
/**
* Applies a Map transformation on a {@link DataStream}. The transformation
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2c4e195c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index 73f8d3a..bf9057b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -54,6 +54,7 @@ import org.apache.flink.streaming.api.streamcomponent.StreamSink;
import org.apache.flink.streaming.api.streamcomponent.StreamSource;
import org.apache.flink.streaming.api.streamcomponent.StreamTask;
import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
+import org.apache.flink.streaming.partitioner.DistributePartitioner;
import org.apache.flink.streaming.partitioner.FieldsPartitioner;
import org.apache.flink.streaming.partitioner.ForwardPartitioner;
import org.apache.flink.streaming.partitioner.GlobalPartitioner;
@@ -573,6 +574,24 @@ public class JobGraphBuilder {
setEdge(upStreamComponentName, downStreamComponentName, new ForwardPartitioner<T>(),
typeNumber);
}
+
+ /**
+ * Connects two components with the given names by distribute partitioning.
+ * <p>
+ * Distribute partitioning: sends the output tuples evenly distributed
+ * along the selected channels
+ *
+ * @param inputStream
+ * The DataStream object of the input
+ * @param upStreamComponentName
+ * Name of the upstream component, that will emit the tuples
+ * @param downStreamComponentName
+ * Name of the downstream component, that will receive the tuples
+ */
+ public <T extends Tuple> void distributeConnect(DataStream<T> inputStream,
+ String upStreamComponentName, String downStreamComponentName) {
+ setEdge(upStreamComponentName, downStreamComponentName, new DistributePartitioner<T>());
+ }
/**
* Connects to JobGraph components with the given names, partitioning and
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2c4e195c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index 9b1a16a..b69c457 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -72,7 +72,7 @@ public abstract class StreamExecutionEnvironment {
* Partitioning strategy on the stream.
*/
public static enum ConnectionType {
- SHUFFLE, BROADCAST, FIELD, FORWARD
+ SHUFFLE, BROADCAST, FIELD, FORWARD, DISTRIBUTE
}
// --------------------------------------------------------------------------------------------
@@ -571,6 +571,9 @@ public abstract class StreamExecutionEnvironment {
case FORWARD:
jobGraphBuilder.forwardConnect(inputStream, input, outputID, typeNumber);
break;
+ case DISTRIBUTE:
+ jobGraphBuilder.distributeConnect(inputStream, input, outputID);
+ break;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2c4e195c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/DistributePartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/DistributePartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/DistributePartitioner.java
new file mode 100644
index 0000000..025106a
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/DistributePartitioner.java
@@ -0,0 +1,51 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.partitioner;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+
+/**
+ * Partitioner that distributes the data equally by cycling through the output
+ * channels.
+ *
+ * @param <T>
+ * Type of the Tuple
+ */
+public class DistributePartitioner<T extends Tuple> implements StreamPartitioner<T> {
+ private static final long serialVersionUID = 1L;
+
+ private int currentChannelIndex;
+ private int[] returnArray;
+
+ public DistributePartitioner() {
+ this.currentChannelIndex = 0;
+ this.returnArray = new int[1];
+ }
+
+ @Override
+ public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
+ int numberOfOutputChannels) {
+ returnArray[0] = currentChannelIndex;
+ currentChannelIndex = (currentChannelIndex + 1) % numberOfOutputChannels;
+ return returnArray;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2c4e195c/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/DistributePartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/DistributePartitionerTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/DistributePartitionerTest.java
new file mode 100644
index 0000000..0a0494c
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/DistributePartitionerTest.java
@@ -0,0 +1,58 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.partitioner;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.junit.Before;
+import org.junit.Test;
+
+public class DistributePartitionerTest {
+
+ private DistributePartitioner<Tuple> distributePartitioner;
+ private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>();
+ private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
+ null);
+
+ @Before
+ public void setPartitioner() {
+ distributePartitioner = new DistributePartitioner<Tuple>();
+ }
+
+ @Test
+ public void testSelectChannelsLength() {
+ sd.setInstance(streamRecord);
+ assertEquals(1, distributePartitioner.selectChannels(sd, 1).length);
+ assertEquals(1, distributePartitioner.selectChannels(sd, 2).length);
+ assertEquals(1, distributePartitioner.selectChannels(sd, 1024).length);
+ }
+
+ @Test
+ public void testSelectChannelsInterval() {
+ sd.setInstance(streamRecord);
+ assertEquals(0, distributePartitioner.selectChannels(sd, 3)[0]);
+ assertEquals(1, distributePartitioner.selectChannels(sd, 3)[0]);
+ assertEquals(2, distributePartitioner.selectChannels(sd, 3)[0]);
+ assertEquals(0, distributePartitioner.selectChannels(sd, 3)[0]);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2c4e195c/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/ForwardPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/ForwardPartitionerTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/ForwardPartitionerTest.java
new file mode 100644
index 0000000..585271f
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/ForwardPartitionerTest.java
@@ -0,0 +1,57 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.partitioner;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ForwardPartitionerTest {
+
+ private ForwardPartitioner<Tuple> forwardPartitioner;
+ private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>();
+ private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
+ null);
+
+ @Before
+ public void setPartitioner() {
+ forwardPartitioner = new ForwardPartitioner<Tuple>();
+ }
+
+ @Test
+ public void testSelectChannelsLength() {
+ sd.setInstance(streamRecord);
+ assertEquals(1, forwardPartitioner.selectChannels(sd, 1).length);
+ assertEquals(1, forwardPartitioner.selectChannels(sd, 2).length);
+ assertEquals(1, forwardPartitioner.selectChannels(sd, 1024).length);
+ }
+
+ @Test
+ public void testSelectChannelsInterval() {
+ sd.setInstance(streamRecord);
+ assertEquals(0, forwardPartitioner.selectChannels(sd, 1)[0]);
+ assertEquals(0, forwardPartitioner.selectChannels(sd, 2)[0]);
+ assertEquals(0, forwardPartitioner.selectChannels(sd, 1024)[0]);
+ }
+}
[03/51] [abbrv] git commit: [streaming] windowReduce added +
BatchReduceInvokable reworked
Posted by se...@apache.org.
[streaming] windowReduce added + BatchReduceInvokable reworked
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/6e521955
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/6e521955
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/6e521955
Branch: refs/heads/master
Commit: 6e521955d8477946d780f36ea9a11180f08a86e1
Parents: 5dbf815
Author: gyfora <gy...@gmail.com>
Authored: Tue Jul 22 17:38:50 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:14:13 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 22 +++
.../api/invokable/operator/BatchIterator.java | 7 +
.../operator/BatchReduceInvokable.java | 133 +++++++++++++++----
3 files changed, 134 insertions(+), 28 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6e521955/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index e67ac0e..4356795 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -388,6 +388,28 @@ public class DataStream<T extends Tuple> {
}
/**
+ * Applies a reduce transformation on preset "time" chunks of the
+ * DataStream. The transformation calls a {@link GroupReduceFunction} on
+ * records received during the predefined time window. The window shifted
+ * after each reduce call. Each GroupReduceFunction call can return any
+ * number of elements including none.
+ *
+ *
+ * @param reducer
+ * The GroupReduceFunction that is called for each time window.
+ * @param windowSize
+ * The time window to run the reducer on, in milliseconds.
+ * @param <R>
+ * output type
+ * @return The modified DataStream.
+ */
+ public <R extends Tuple> StreamOperator<T, R> windowReduce(GroupReduceFunction<T, R> reducer,
+ long windowSize) {
+ return environment.addFunction("batchReduce", new DataStream<T>(this), reducer,
+ new BatchReduceInvokable<T, R>(reducer, windowSize));
+ }
+
+ /**
* Adds the given sink to this environment. Only streams with sinks added
* will be executed once the {@link StreamExecutionEnvironment#execute()}
* method is called.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6e521955/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
new file mode 100755
index 0000000..1432749
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
@@ -0,0 +1,7 @@
+package org.apache.flink.streaming.api.invokable.operator;
+
+import java.util.Iterator;
+
+public interface BatchIterator<IN> extends Iterator<IN> {
+ public void reset();
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6e521955/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 811a929..2d2d890 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -20,73 +20,149 @@
package org.apache.flink.streaming.api.invokable.operator;
import java.io.IOException;
-import java.util.Iterator;
import org.apache.flink.api.java.functions.GroupReduceFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
- private int counter;
private int batchSize;
+ private long windowSize;
+ volatile boolean isRunning;
+ boolean window;
private GroupReduceFunction<IN, OUT> reducer;
public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, int batchSize) {
this.reducer = reduceFunction;
- this.counter = 0;
this.batchSize = batchSize;
}
+ public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize) {
+ this.reducer = reduceFunction;
+ this.windowSize = windowSize;
+ this.window = true;
+ }
+
@Override
public void invoke() throws Exception {
- MyIterator it = new MyIterator();
- if (this.isMutable) {
- do {
- reducer.reduce(it, collector);
- it.reset();
- } while (reuse != null);
+ BatchIterator<IN> userIterator;
+ if (window) {
+ userIterator = new WindowIterator();
} else {
- do {
- reducer.reduce(it, collector);
- it.reset();
- } while (reuse != null);
+ userIterator = new CounterIterator();
}
+ do {
+ if (userIterator.hasNext()) {
+ reducer.reduce(userIterator, collector);
+ userIterator.reset();
+ }
+ } while (reuse != null);
+ }
+
+ private StreamRecord<IN> loadNextRecord() {
+ if (!isMutable) {
+ resetReuse();
+ }
+ try {
+ reuse = recordIterator.next(reuse);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ return reuse;
}
- public class MyIterator implements Iterator<IN> {
+ public class CounterIterator implements BatchIterator<IN> {
+ private int counter;
+ private boolean loadedNext;
- public MyIterator() {
- reset();
+ public CounterIterator() {
+ counter = 1;
}
@Override
public boolean hasNext() {
-
- if (counter >= batchSize) {
+ if (counter > batchSize) {
return false;
+ } else if (!loadedNext) {
+ loadNextRecord();
+ loadedNext = true;
+ }
+ return (reuse != null);
+ }
+
+ @Override
+ public IN next() {
+ if (hasNext()) {
+ counter++;
+ loadedNext = false;
+ return reuse.getTuple();
} else {
- try {
- resetReuse();
- reuse = recordIterator.next(reuse);
- } catch (IOException e) {
- e.printStackTrace();
- }
- return (reuse != null);
+ counter++;
+ loadedNext = false;
+ return null;
+ }
+ }
+
+ public void reset() {
+ for (int i = 0; i < (batchSize - counter); i++) {
+ loadNextRecord();
}
+ loadNextRecord();
+ loadedNext = true;
+ counter = 1;
+ }
+
+ @Override
+ public void remove() {
+
+ }
+
+ }
+
+ public class WindowIterator implements BatchIterator<IN> {
+
+ volatile boolean iterate;
+ private boolean loadedNext;
+ private long startTime;
+
+ public WindowIterator() {
+ startTime = System.currentTimeMillis();
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (System.currentTimeMillis() - startTime > windowSize) {
+ return false;
+ } else if (!loadedNext) {
+ loadNextRecord();
+ loadedNext = true;
+ }
+ return (reuse != null);
}
@Override
public IN next() {
- counter++;
- return reuse.getTuple();
+ if (hasNext()) {
+ loadedNext = false;
+ return reuse.getTuple();
+ } else {
+ loadedNext = false;
+ return reuse.getTuple();
+ }
}
public void reset() {
- counter = 0;
+ while (System.currentTimeMillis() - startTime < windowSize) {
+ loadNextRecord();
+ }
+ loadNextRecord();
+ loadedNext = true;
+ startTime = System.currentTimeMillis();
}
@Override
@@ -95,4 +171,5 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
}
}
+
}
\ No newline at end of file
[19/51] [abbrv] git commit: [streaming] connectors logging and error
handling fixed
Posted by se...@apache.org.
[streaming] connectors logging and error handling fixed
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/7c5bc3c5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/7c5bc3c5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/7c5bc3c5
Branch: refs/heads/master
Commit: 7c5bc3c5f5323de4a73aaa8f6864a441e64a00ef
Parents: a06f9d1
Author: jfeher <fe...@gmail.com>
Authored: Tue Jul 29 16:20:12 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:21:13 2014 +0200
----------------------------------------------------------------------
.../streaming/connectors/flume/FlumeSink.java | 29 +++++++++------
.../streaming/connectors/flume/FlumeSource.java | 39 ++++++++++----------
.../connectors/flume/FlumeTopology.java | 7 ++--
.../streaming/connectors/kafka/KafkaSink.java | 3 +-
.../connectors/kafka/KafkaTopology.java | 16 +++-----
.../streaming/connectors/rabbitmq/RMQSink.java | 12 +++++-
.../connectors/rabbitmq/RMQSource.java | 24 ++++++------
.../connectors/rabbitmq/RMQTopology.java | 5 +--
8 files changed, 72 insertions(+), 63 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7c5bc3c5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
index 27ce546..3379fcb 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
@@ -19,19 +19,23 @@
package org.apache.flink.streaming.connectors.flume;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.flume.Event;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.FlumeException;
import org.apache.flume.api.RpcClient;
import org.apache.flume.api.RpcClientFactory;
import org.apache.flume.event.EventBuilder;
-
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.connectors.rabbitmq.RMQSource;
public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
+ private static final Log LOG = LogFactory.getLog(RMQSource.class);
+
private transient FlinkRpcClientFacade client;
boolean initDone = false;
String host;
@@ -53,7 +57,7 @@ public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
}
byte[] data = serialize(tuple);
- if(!closeWithoutSend){
+ if (!closeWithoutSend) {
client.sendDataToFlume(data);
}
if (sendAndClose) {
@@ -74,22 +78,23 @@ public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
this.hostname = hostname;
this.port = port;
int initCounter = 0;
- while(true){
- if(initCounter >= 90){
+ while (true) {
+ if (initCounter >= 90) {
System.exit(1);
}
- try{
+ try {
this.client = RpcClientFactory.getDefaultInstance(hostname, port);
- }
- catch(FlumeException e){
+ } catch (FlumeException e) {
// Wait one second if the connection failed before the next try
try {
Thread.sleep(1000);
} catch (InterruptedException e1) {
- e1.printStackTrace();
+ if (LOG.isErrorEnabled()) {
+ LOG.error("Interrupted while trying to connect " + port + " at " + host);
+ }
}
}
- if(client != null){
+ if (client != null) {
break;
}
initCounter++;
@@ -120,10 +125,10 @@ public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
public void sendAndClose() {
sendAndClose = true;
}
-
- public void closeWithoutSend(){
+
+ public void closeWithoutSend() {
client.close();
- closeWithoutSend=true;
+ closeWithoutSend = true;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7c5bc3c5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
index 966c881..5b90e14 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
@@ -36,15 +36,15 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
String host;
String port;
-
- FlumeSource(String host, int port){
+
+ FlumeSource(String host, int port) {
this.host = host;
this.port = Integer.toString(port);
}
-
+
public class MyAvroSource extends AvroSource {
Collector<IN> collector;
-
+
@Override
public Status append(AvroFlumeEvent avroEvent) {
collect(avroEvent);
@@ -59,30 +59,31 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
return Status.OK;
}
-
- private void collect(AvroFlumeEvent avroEvent){
+
+ private void collect(AvroFlumeEvent avroEvent) {
byte[] b = avroEvent.getBody().array();
IN tuple = FlumeSource.this.deserialize(b);
- if(!closeWithoutSend){
+ if (!closeWithoutSend) {
collector.collect(tuple);
}
- if(sendAndClose){
+ if (sendAndClose) {
sendDone = true;
}
}
-
+
}
MyAvroSource avroSource;
private volatile boolean closeWithoutSend = false;
private boolean sendAndClose = false;
private volatile boolean sendDone = false;
+
public abstract IN deserialize(byte[] msg);
-
- public void configureAvroSource(Collector<IN> collector){
+
+ public void configureAvroSource(Collector<IN> collector) {
avroSource = new MyAvroSource();
- avroSource.collector=collector;
+ avroSource.collector = collector;
Context context = new Context();
context.put("port", port);
context.put("bind", host);
@@ -90,24 +91,24 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
ChannelProcessor cp = new ChannelProcessor(null);
avroSource.setChannelProcessor(cp);
}
-
+
@Override
public void invoke(Collector<IN> collector) throws Exception {
configureAvroSource(collector);
avroSource.start();
- while(true){
- if(closeWithoutSend || sendDone){
+ while (true) {
+ if (closeWithoutSend || sendDone) {
break;
}
}
avroSource.stop();
}
-
- public void sendAndClose(){
+
+ public void sendAndClose() {
sendAndClose = true;
}
-
- public void closeWithoutSend(){
+
+ public void closeWithoutSend() {
closeWithoutSend = true;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7c5bc3c5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
index e08837a..779a5fb 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
@@ -20,7 +20,6 @@
package org.apache.flink.streaming.connectors.flume;
import org.apache.commons.lang.SerializationUtils;
-
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
@@ -40,7 +39,8 @@ public class FlumeTopology {
try {
sendAndClose();
} catch (Exception e) {
- e.printStackTrace();
+ new RuntimeException("Error while closing RMQ connection with " + port + " at "
+ + host, e);
}
}
return SerializationUtils.serialize((String) tuple.getField(0));
@@ -70,8 +70,7 @@ public class FlumeTopology {
public static void main(String[] args) throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment
- .createLocalEnvironment(1);
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
@SuppressWarnings("unused")
DataStream<Tuple1<String>> dataStream1 = env
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7c5bc3c5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
index e97cdd0..44867ef 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
@@ -64,8 +64,7 @@ public abstract class KafkaSink<IN extends Tuple, OUT> extends SinkFunction<IN>
}
OUT out = serialize(tuple);
- KeyedMessage<Integer, OUT> data = new KeyedMessage<Integer, OUT>(
- topicId, out);
+ KeyedMessage<Integer, OUT> data = new KeyedMessage<Integer, OUT>(topicId, out);
if (!closeWithoutSend) {
producer.send(data);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7c5bc3c5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
index e111feb..d605fb8 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
@@ -31,8 +31,7 @@ public class KafkaTopology {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Collector<Tuple1<String>> collector)
- throws Exception {
+ public void invoke(Collector<Tuple1<String>> collector) throws Exception {
for (int i = 0; i < 10; i++) {
collector.collect(new Tuple1<String>(Integer.toString(i)));
}
@@ -44,8 +43,7 @@ public class KafkaTopology {
public static final class MyKafkaSource extends KafkaSource<Tuple1<String>> {
private static final long serialVersionUID = 1L;
- public MyKafkaSource(String zkQuorum, String groupId, String topicId,
- int numThreads) {
+ public MyKafkaSource(String zkQuorum, String groupId, String topicId, int numThreads) {
super(zkQuorum, groupId, topicId, numThreads);
}
@@ -60,8 +58,7 @@ public class KafkaTopology {
}
- public static final class MyKafkaSink extends
- KafkaSink<Tuple1<String>, String> {
+ public static final class MyKafkaSink extends KafkaSink<Tuple1<String>, String> {
private static final long serialVersionUID = 1L;
public MyKafkaSink(String topicId, String brokerAddr) {
@@ -82,19 +79,18 @@ public class KafkaTopology {
public static void main(String[] args) {
- StreamExecutionEnvironment env = StreamExecutionEnvironment
- .createLocalEnvironment(1);
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
@SuppressWarnings("unused")
DataStream<Tuple1<String>> stream1 = env
- .addSource(new MyKafkaSource("localhost:2181", "group", "test", 1), SOURCE_PARALELISM)
+ .addSource(new MyKafkaSource("localhost:2181", "group", "test", 1), SOURCE_PARALELISM)
.print();
@SuppressWarnings("unused")
DataStream<Tuple1<String>> stream2 = env
.addSource(new MySource())
.addSink(new MyKafkaSink("test", "localhost:9092"));
-
+
env.execute();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7c5bc3c5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index 5440ea7..663fc13 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -25,11 +25,16 @@ import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
import com.rabbitmq.client.ConnectionFactory;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
+
+ private static final Log LOG = LogFactory.getLog(RMQSource.class);
+
private boolean sendAndClose = false;
private boolean closeWithoutSend = false;
@@ -72,7 +77,9 @@ public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
channel.basicPublish("", QUEUE_NAME, null, msg);
}
} catch (IOException e) {
- e.printStackTrace();
+ if (LOG.isErrorEnabled()) {
+ LOG.error("Cannot send RMQ message " + QUEUE_NAME + " at " + HOST_NAME);
+ }
}
if (sendAndClose) {
@@ -87,7 +94,8 @@ public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
channel.close();
connection.close();
} catch (IOException e) {
- e.printStackTrace();
+ new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME + " at "
+ + HOST_NAME, e);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7c5bc3c5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index 345d884..4fd2235 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -21,12 +21,13 @@ package org.apache.flink.streaming.connectors.rabbitmq;
import java.io.IOException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
import com.rabbitmq.client.ConnectionFactory;
-import com.rabbitmq.client.ConsumerCancelledException;
import com.rabbitmq.client.QueueingConsumer;
-import com.rabbitmq.client.ShutdownSignalException;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.function.source.SourceFunction;
@@ -35,6 +36,8 @@ import org.apache.flink.util.Collector;
public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
private static final long serialVersionUID = 1L;
+ private static final Log LOG = LogFactory.getLog(RMQSource.class);
+
private final String QUEUE_NAME;
private final String HOST_NAME;
private boolean closeWithoutSend = false;
@@ -63,6 +66,8 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
consumer = new QueueingConsumer(channel);
channel.basicConsume(QUEUE_NAME, true, consumer);
} catch (IOException e) {
+ new RuntimeException("Cannot create RMQ connection with " + QUEUE_NAME + " at "
+ + HOST_NAME, e);
}
}
@@ -74,14 +79,10 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
try {
delivery = consumer.nextDelivery();
- } catch (ShutdownSignalException e) {
- e.printStackTrace();
- break;
- } catch (ConsumerCancelledException e) {
- e.printStackTrace();
- break;
- } catch (InterruptedException e) {
- e.printStackTrace();
+ } catch (Exception e) {
+ if (LOG.isErrorEnabled()) {
+ LOG.error("Cannot receive RMQ message " + QUEUE_NAME + " at " + HOST_NAME);
+ }
}
outTuple = deserialize(delivery.getBody());
@@ -98,7 +99,8 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
try {
connection.close();
} catch (IOException e) {
- e.printStackTrace();
+ new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME + " at "
+ + HOST_NAME, e);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7c5bc3c5/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
index f84505b..828c2fa 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
@@ -68,8 +68,7 @@ public class RMQTopology {
public static void main(String[] args) throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment
- .createLocalEnvironment(1);
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
@SuppressWarnings("unused")
DataStream<Tuple1<String>> dataStream1 = env
@@ -78,7 +77,7 @@ public class RMQTopology {
@SuppressWarnings("unused")
DataStream<Tuple1<String>> dataStream2 = env
- .fromElements("one", "two", "three", "four", "five", "q")
+ .fromElements("one", "two", "three", "four", "five", "q")
.addSink(new MyRMQSink("localhost", "hello"));
env.execute();
[43/51] [abbrv] git commit: [streaming] Added groupBy operator to
DataStream + name refactor
Posted by se...@apache.org.
[streaming] Added groupBy operator to DataStream + name refactor
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/696bce02
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/696bce02
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/696bce02
Branch: refs/heads/master
Commit: 696bce02f42973777304b0ad3fe4cc62a4f5e976
Parents: e089959
Author: gyfora <gy...@gmail.com>
Authored: Fri Aug 8 13:32:55 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:43 2014 +0200
----------------------------------------------------------------------
.../streaming/api/datastream/CoDataStream.java | 121 --------------
.../api/datastream/ConnectedDataStream.java | 119 +++++++++-----
.../streaming/api/datastream/DataStream.java | 87 +++++-----
.../api/datastream/GroupedDataStream.java | 162 +++++++++++++++++++
.../api/datastream/IterativeDataStream.java | 4 +-
.../api/datastream/MergedDataStream.java | 84 ++++++++++
.../operator/BatchGroupReduceInvokable.java | 70 ++++++++
.../operator/BatchReduceInvokable.java | 88 ++++------
.../operator/GroupReduceInvokable.java | 8 +-
.../operator/StreamReduceInvokable.java | 46 ++++++
.../operator/WindowGroupReduceInvokable.java | 67 ++++++++
.../operator/WindowReduceInvokable.java | 71 ++++----
.../streaming/state/MutableTableState.java | 8 +-
.../streaming/state/SlidingWindowState.java | 20 ++-
.../state/SlidingWindowStateIterator.java | 40 ++++-
.../flink/streaming/state/TableState.java | 6 +-
.../apache/flink/streaming/api/PrintTest.java | 5 +-
.../api/invokable/operator/CoMapTest.java | 4 +-
.../api/invokable/operator/MapTest.java | 4 +-
.../streaming/state/SlidingWindowStateTest.java | 12 +-
.../examples/cellinfo/CellInfoLocal.java | 2 +-
.../streaming/examples/join/JoinLocal.java | 2 +-
.../ml/IncrementalLearningSkeleton.java | 2 +-
.../streaming/examples/ml/IncrementalOLS.java | 2 +-
.../examples/window/join/WindowJoinLocal.java | 2 +-
.../examples/wordcount/WordCountLocal.java | 3 +-
26 files changed, 685 insertions(+), 354 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
deleted file mode 100755
index b974b1d..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import java.io.Serializable;
-
-import org.apache.commons.lang3.SerializationException;
-import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.streaming.api.JobGraphBuilder;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.co.CoMapFunction;
-import org.apache.flink.streaming.api.function.co.RichCoMapFunction;
-import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
-import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
-import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
-
-/**
- * The CoDataStream represents a stream for two different data types. It can be
- * used to apply transformations like {@link CoMapFunction} on two
- * {@link DataStream}s
- *
- * @param <IN1>
- * Type of the first DataSteam.
- * @param <IN2>
- * Type of the second DataStream.
- */
-public class CoDataStream<IN1, IN2> {
-
- StreamExecutionEnvironment environment;
- JobGraphBuilder jobGraphBuilder;
- DataStream<IN1> input1;
- DataStream<IN2> input2;
-
- protected CoDataStream(StreamExecutionEnvironment environment, JobGraphBuilder jobGraphBuilder,
- DataStream<IN1> input1, DataStream<IN2> input2) {
- this.jobGraphBuilder = jobGraphBuilder;
- this.environment = environment;
- this.input1 = input1.copy();
- this.input2 = input2.copy();
- }
-
- /**
- * Returns the first {@link DataStream}.
- *
- * @return The first DataStream.
- */
- public DataStream<IN1> getFirst() {
- return input1.copy();
- }
-
- /**
- * Returns the second {@link DataStream}.
- *
- * @return The second DataStream.
- */
- public DataStream<IN2> getSecond() {
- return input2.copy();
- }
-
- /**
- * Applies a CoMap transformation on two separate {@link DataStream}s. The
- * transformation calls a {@link CoMapFunction#map1} for each element of the
- * first input and {@link CoMapFunction#map2} for each element of the second
- * input. Each CoMapFunction call returns exactly one element. The user can
- * also extend {@link RichCoMapFunction} to gain access to other features
- * provided by the {@link RichFuntion} interface.
- *
- * @param coMapper
- * The CoMapFunction used to jointly transform the two input
- * DataStreams
- * @return The transformed DataStream
- */
- public <OUT> SingleOutputStreamOperator<OUT, ?> map(CoMapFunction<IN1, IN2, OUT> coMapper) {
- return addCoFunction("coMap", coMapper, new FunctionTypeWrapper<IN1, IN2, OUT>(coMapper,
- CoMapFunction.class, 0, 1, 2), new CoMapInvokable<IN1, IN2, OUT>(coMapper));
- }
-
- protected <OUT> SingleOutputStreamOperator<OUT, ?> addCoFunction(String functionName,
- final Function function, TypeSerializerWrapper<IN1, IN2, OUT> typeWrapper,
- CoInvokable<IN1, IN2, OUT> functionInvokable) {
-
- @SuppressWarnings({ "unchecked", "rawtypes" })
- SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(
- environment, functionName);
-
- try {
- input1.jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, typeWrapper,
- functionName, SerializationUtils.serialize((Serializable) function),
- environment.getDegreeOfParallelism());
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize user defined function");
- }
-
- input1.connectGraph(input1, returnStream.getId(), 1);
- input1.connectGraph(input2, returnStream.getId(), 2);
-
- // TODO consider iteration
-
- return returnStream;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
index 1d8fb48..16a383f 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
@@ -19,66 +19,103 @@
package org.apache.flink.streaming.api.datastream;
-import java.util.ArrayList;
-import java.util.List;
+import java.io.Serializable;
+import org.apache.commons.lang3.SerializationException;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.streaming.api.JobGraphBuilder;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.api.function.co.CoMapFunction;
+import org.apache.flink.streaming.api.function.co.RichCoMapFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
+import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
+import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
/**
- * The ConnectedDataStream represents a DataStream which consists of connected
- * outputs of DataStreams of the same type. Operators applied on this will
- * transform all the connected outputs jointly.
+ * The ConnectedDataStream represents a stream for two different data types. It can be
+ * used to apply transformations like {@link CoMapFunction} on two
+ * {@link DataStream}s
*
- * @param <OUT>
- * Type of the output.
+ * @param <IN1>
+ * Type of the first DataSteam.
+ * @param <IN2>
+ * Type of the second DataStream.
*/
-public class ConnectedDataStream<OUT> extends DataStream<OUT> {
+public class ConnectedDataStream<IN1, IN2> {
- protected List<DataStream<OUT>> connectedStreams;
+ StreamExecutionEnvironment environment;
+ JobGraphBuilder jobGraphBuilder;
+ DataStream<IN1> input1;
+ DataStream<IN2> input2;
- protected ConnectedDataStream(StreamExecutionEnvironment environment, String operatorType) {
- super(environment, operatorType);
- this.connectedStreams = new ArrayList<DataStream<OUT>>();
- this.connectedStreams.add(this);
+ protected ConnectedDataStream(StreamExecutionEnvironment environment, JobGraphBuilder jobGraphBuilder,
+ DataStream<IN1> input1, DataStream<IN2> input2) {
+ this.jobGraphBuilder = jobGraphBuilder;
+ this.environment = environment;
+ this.input1 = input1.copy();
+ this.input2 = input2.copy();
}
- protected ConnectedDataStream(DataStream<OUT> dataStream) {
- super(dataStream);
- connectedStreams = new ArrayList<DataStream<OUT>>();
- if (dataStream instanceof ConnectedDataStream) {
- for (DataStream<OUT> stream : ((ConnectedDataStream<OUT>) dataStream).connectedStreams) {
- connectedStreams.add(stream);
- }
- } else {
- this.connectedStreams.add(this);
- }
-
+ /**
+ * Returns the first {@link DataStream}.
+ *
+ * @return The first DataStream.
+ */
+ public DataStream<IN1> getFirst() {
+ return input1.copy();
}
- // @Override
- // public IterativeDataStream<OUT> iterate() {
- // throw new RuntimeException("Cannot iterate connected DataStreams");
- // }
+ /**
+ * Returns the second {@link DataStream}.
+ *
+ * @return The second DataStream.
+ */
+ public DataStream<IN2> getSecond() {
+ return input2.copy();
+ }
- protected void addConnection(DataStream<OUT> stream) {
- connectedStreams.add(stream.copy());
+ /**
+ * Applies a CoMap transformation on two separate {@link DataStream}s. The
+ * transformation calls a {@link CoMapFunction#map1} for each element of the
+ * first input and {@link CoMapFunction#map2} for each element of the second
+ * input. Each CoMapFunction call returns exactly one element. The user can
+ * also extend {@link RichCoMapFunction} to gain access to other features
+ * provided by the {@link RichFuntion} interface.
+ *
+ * @param coMapper
+ * The CoMapFunction used to jointly transform the two input
+ * DataStreams
+ * @return The transformed DataStream
+ */
+ public <OUT> SingleOutputStreamOperator<OUT, ?> map(CoMapFunction<IN1, IN2, OUT> coMapper) {
+ return addCoFunction("coMap", coMapper, new FunctionTypeWrapper<IN1, IN2, OUT>(coMapper,
+ CoMapFunction.class, 0, 1, 2), new CoMapInvokable<IN1, IN2, OUT>(coMapper));
}
- @Override
- protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
- ConnectedDataStream<OUT> returnStream = (ConnectedDataStream<OUT>) this.copy();
+ protected <OUT> SingleOutputStreamOperator<OUT, ?> addCoFunction(String functionName,
+ final Function function, TypeSerializerWrapper<IN1, IN2, OUT> typeWrapper,
+ CoInvokable<IN1, IN2, OUT> functionInvokable) {
+
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(
+ environment, functionName);
- for (DataStream<OUT> stream : returnStream.connectedStreams) {
- stream.partitioner = partitioner;
+ try {
+ input1.jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, typeWrapper,
+ functionName, SerializationUtils.serialize((Serializable) function),
+ environment.getDegreeOfParallelism());
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize user defined function");
}
- return returnStream;
- }
+ input1.connectGraph(input1, returnStream.getId(), 1);
+ input1.connectGraph(input2, returnStream.getId(), 2);
- @Override
- protected ConnectedDataStream<OUT> copy() {
- return new ConnectedDataStream<OUT>(this);
+ // TODO consider iteration
+
+ return returnStream;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 531f43c..9f802f7 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -30,12 +30,10 @@ import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.GroupReduceFunction;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.java.functions.RichFilterFunction;
import org.apache.flink.api.java.functions.RichFlatMapFunction;
import org.apache.flink.api.java.functions.RichGroupReduceFunction;
import org.apache.flink.api.java.functions.RichMapFunction;
-import org.apache.flink.api.java.functions.RichReduceFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.JobGraphBuilder;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -50,7 +48,6 @@ import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
-import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
import org.apache.flink.streaming.api.invokable.operator.WindowReduceInvokable;
import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
@@ -154,16 +151,16 @@ public abstract class DataStream<OUT> {
}
/**
- * Creates a new by connecting {@link DataStream} outputs of the same type
- * with each other. The DataStreams connected using this operator will be
- * transformed simultaneously.
+ * Creates a new {@link MergedDataStream} by merging {@link DataStream}
+ * outputs of the same type with each other. The DataStreams merged using
+ * this operator will be transformed simultaneously.
*
* @param streams
- * The DataStreams to connect output with.
- * @return The {@link ConnectedDataStream}.
+ * The DataStreams to merge output with.
+ * @return The {@link MergedDataStream}.
*/
- public ConnectedDataStream<OUT> connectWith(DataStream<OUT>... streams) {
- ConnectedDataStream<OUT> returnStream = new ConnectedDataStream<OUT>(this);
+ public MergedDataStream<OUT> merge(DataStream<OUT>... streams) {
+ MergedDataStream<OUT> returnStream = new MergedDataStream<OUT>(this);
for (DataStream<OUT> stream : streams) {
returnStream.addConnection(stream);
@@ -172,16 +169,16 @@ public abstract class DataStream<OUT> {
}
/**
- * Creates a new {@link CoDataStream} by connecting {@link DataStream}
- * outputs of different type with each other. The DataStreams connected
- * using this operators can be used with CoFunctions.
+ * Creates a new {@link ConnectedDataStream} by connecting
+ * {@link DataStream} outputs of different type with each other. The
+ * DataStreams connected using this operators can be used with CoFunctions.
*
* @param dataStream
* The DataStream with which this stream will be joined.
- * @return The {@link CoDataStream}.
+ * @return The {@link ConnectedDataStream}.
*/
- public <R> CoDataStream<OUT, R> co(DataStream<R> dataStream) {
- return new CoDataStream<OUT, R>(environment, jobGraphBuilder, this, dataStream);
+ public <R> ConnectedDataStream<OUT, R> connect(DataStream<R> dataStream) {
+ return new ConnectedDataStream<OUT, R>(environment, jobGraphBuilder, this, dataStream);
}
/**
@@ -282,27 +279,8 @@ public abstract class DataStream<OUT> {
flatMapper));
}
- /**
- * Applies a group and a reduce transformation on the DataStream grouped on
- * the given key position. The {@link ReduceFunction} will receive input
- * values based on the key value. Only input values with the same key will
- * go to the same reducer.The user can also extend
- * {@link RichReduceFunction} to gain access to other features provided
- * by the {@link RichFuntion} interface.
- *
- * @param reducer
- * The {@link ReduceFunction} that will be called for every
- * element of the input values with the same key.
- * @param keyPosition
- * The key position in the input values on which the grouping is
- * made.
- * @return The transformed DataStream.
- */
- public SingleOutputStreamOperator<OUT, ?> groupReduce(ReduceFunction<OUT> reducer,
- int keyPosition) {
- return addFunction("groupReduce", reducer,
- new FunctionTypeWrapper<OUT, Tuple, OUT>(reducer, ReduceFunction.class, 0, -1, 0),
- new GroupReduceInvokable<OUT>(reducer, keyPosition)).partitionBy(keyPosition);
+ public GroupedDataStream<OUT> groupBy(int keyPosition) {
+ return new GroupedDataStream<OUT>(this.partitionBy(keyPosition), keyPosition);
}
/**
@@ -359,7 +337,7 @@ public abstract class DataStream<OUT> {
/**
* Applies a reduce transformation on preset "time" chunks of the
* DataStream. The transformation calls a {@link GroupReduceFunction} on
- * records received during the predefined time window. The window shifted
+ * records received during the predefined time window. The window is shifted
* after each reduce call. Each GroupReduceFunction call can return any
* number of elements including none.The user can also extend
* {@link RichGroupReduceFunction} to gain access to other features provided
@@ -379,14 +357,35 @@ public abstract class DataStream<OUT> {
long windowSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<OUT, R>(reducer,
- windowSize, windowSize, windowSize));
+ windowSize, windowSize));
}
+ /**
+ * Applies a reduce transformation on preset "time" chunks of the
+ * DataStream. The transformation calls a {@link GroupReduceFunction} on
+ * records received during the predefined time window. The window is shifted
+ * after each reduce call. Each GroupReduceFunction call can return any
+ * number of elements including none.The user can also extend
+ * {@link RichGroupReduceFunction} to gain access to other features provided
+ * by the {@link RichFuntion} interface.
+ *
+ *
+ * @param reducer
+ * The GroupReduceFunction that is called for each time window.
+ * @param windowSize
+ * SingleOutputStreamOperator The time window to run the reducer
+ * on, in milliseconds.
+ * @param slideInterval
+ * The time interval, batch is slid by.
+ * @param <R>
+ * output type
+ * @return The transformed DataStream.
+ */
public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
- long windowSize, long slideInterval, long timeUnitInMillis) {
+ long windowSize, long slideInterval) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT, Tuple, R>(reducer,
GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<OUT, R>(reducer,
- windowSize, slideInterval, timeUnitInMillis));
+ windowSize, slideInterval));
}
/**
@@ -775,7 +774,7 @@ public abstract class DataStream<OUT> {
* type of the return stream
* @return the data stream constructed
*/
- private <R> SingleOutputStreamOperator<R, ?> addFunction(String functionName,
+ protected <R> SingleOutputStreamOperator<R, ?> addFunction(String functionName,
final Function function, TypeSerializerWrapper<OUT, Tuple, R> typeWrapper,
UserTaskInvokable<OUT, R> functionInvokable) {
@@ -832,8 +831,8 @@ public abstract class DataStream<OUT> {
* Number of the type (used at co-functions)
*/
protected <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
- if (inputStream instanceof ConnectedDataStream) {
- for (DataStream<X> stream : ((ConnectedDataStream<X>) inputStream).connectedStreams) {
+ if (inputStream instanceof MergedDataStream) {
+ for (DataStream<X> stream : ((MergedDataStream<X>) inputStream).mergedStreams) {
jobGraphBuilder.setEdge(stream.getId(), outputID, stream.partitioner, typeNumber,
inputStream.userDefinedNames);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
new file mode 100755
index 0000000..0b78018
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
@@ -0,0 +1,162 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.invokable.operator.BatchGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.WindowGroupReduceInvokable;
+import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
+
+/**
+ * A GroupedDataStream represents a data stream which has been partitioned by
+ * the given key in the values. Operators like {@link #reduce},
+ * {@link #batchReduce} etc. can be applied on the {@link GroupedDataStream}.
+ *
+ * @param <OUT>
+ * The output type of the {@link GroupedDataStream}.
+ */
+public class GroupedDataStream<OUT> {
+
+ DataStream<OUT> dataStream;
+ int keyPosition;
+
+ protected GroupedDataStream(DataStream<OUT> dataStream, int keyPosition) {
+ this.dataStream = dataStream.copy();
+ this.keyPosition = keyPosition;
+ }
+
+ /**
+ * Applies a reduce transformation on the grouped data stream grouped on by
+ * the given key position. The {@link ReduceFunction} will receive input
+ * values based on the key value. Only input values with the same key will
+ * go to the same reducer.The user can also extend
+ * {@link RichReduceFunction} to gain access to other features provided by
+ * the {@link RichFuntion} interface.
+ *
+ * @param reducer
+ * The {@link ReduceFunction} that will be called for every
+ * element of the input values with the same key.
+ * @return The transformed DataStream.
+ */
+ public SingleOutputStreamOperator<OUT, ?> reduce(ReduceFunction<OUT> reducer) {
+ return dataStream.addFunction("groupReduce", reducer,
+ new FunctionTypeWrapper<OUT, Tuple, OUT>(reducer, ReduceFunction.class, 0, -1, 0),
+ new GroupReduceInvokable<OUT>(reducer, keyPosition)).partitionBy(keyPosition);
+ }
+
+ /**
+ * Applies a group reduce transformation on preset chunks of the grouped
+ * data stream. The {@link ReduceFunction} will receive input values based
+ * on the key value. Only input values with the same key will go to the same
+ * reducer.When the reducer has ran for all the values in the batch, the
+ * batch is slid forward. access to other features provided by the
+ * {@link RichFuntion} interface.
+ *
+ *
+ * @param reducer
+ * The {@link ReduceFunction} that will be called for every
+ * element of the input values with the same key.
+ * @param batchSize
+ * The size of the data stream chunk (the number of values in the
+ * batch).
+ * @return The transformed {@link DataStream}.
+ */
+ public SingleOutputStreamOperator<OUT, ?> batchReduce(ReduceFunction<OUT> reducer, int batchSize) {
+ return batchReduce(reducer, batchSize, batchSize);
+ }
+
+ /**
+ * Applies a group reduce transformation on preset chunks of the grouped
+ * data stream in a sliding window fashion. The {@link ReduceFunction} will
+ * receive input values based on the key value. Only input values with the
+ * same key will go to the same reducer. When the reducer has ran for all
+ * the values in the batch, the batch is slid forward. The user can also
+ * extend {@link RichReduceFunction} to gain access to other features
+ * provided by the {@link RichFuntion} interface.
+ *
+ * @param reducer
+ * The {@link ReduceFunction} that will be called for every
+ * element of the input values with the same key.
+ * @param batchSize
+ * The size of the data stream chunk (the number of values in the
+ * batch).
+ * @param slideSize
+ * The number of values the batch is slid by.
+ * @return The transformed {@link DataStream}.
+ */
+ public SingleOutputStreamOperator<OUT, ?> batchReduce(ReduceFunction<OUT> reducer,
+ long batchSize, long slideSize) {
+ return dataStream.addFunction("batchReduce", reducer,
+ new FunctionTypeWrapper<OUT, Tuple, OUT>(reducer, ReduceFunction.class, 0, -1,
+ 0), new BatchGroupReduceInvokable<OUT>(reducer, batchSize, slideSize,
+ keyPosition));
+ }
+
+ /**
+ * Applies a group reduce transformation on preset "time" chunks of the
+ * grouped data stream. The {@link ReduceFunction} will receive input values
+ * based on the key value. Only input values with the same key will go to
+ * the same reducer.When the reducer has ran for all the values in the
+ * batch, the window is shifted forward gain access to other features
+ * provided by the {@link RichFuntion} interface.
+ *
+ *
+ * @param reducer
+ * The GroupReduceFunction that is called for each time window.
+ * @param windowSize
+ * SingleOutputStreamOperator The time window to run the reducer
+ * on, in milliseconds.
+ * @return The transformed DataStream.
+ */
+ public SingleOutputStreamOperator<OUT, ?> windowReduce(ReduceFunction<OUT> reducer,
+ long windowSize) {
+ return windowReduce(reducer, windowSize, windowSize, windowSize);
+ }
+
+ /**
+ * Applies a group reduce transformation on preset "time" chunks of the
+ * grouped data stream in a sliding window fashion. The
+ * {@link ReduceFunction} will receive input values based on the key value.
+ * Only input values with the same key will go to the same reducer. When the
+ * reducer has ran for all the values in the batch, the window is shifted
+ * forward. The user can also extend {@link RichReduceFunction} to gain
+ * access to other features provided by the {@link RichFuntion} interface.
+ *
+ * @param reducer
+ * The GroupReduceFunction that is called for each time window.
+ * @param windowSize
+ * SingleOutputStreamOperator The time window to run the reducer
+ * on, in milliseconds.
+ * @param slideSize
+ * The time interval the batch is slid by.
+ * @return The transformed DataStream.
+ */
+ public SingleOutputStreamOperator<OUT, ?> windowReduce(ReduceFunction<OUT> reducer,
+ long windowSize, long slideInterval, long timeUnitInMillis) {
+ return dataStream.addFunction("batchReduce", reducer,
+ new FunctionTypeWrapper<OUT, Tuple, OUT>(reducer, ReduceFunction.class, 0, -1,
+ 0), new WindowGroupReduceInvokable<OUT>(reducer, windowSize, slideInterval,
+ keyPosition));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
index f1cb13c..75ce29a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
@@ -91,8 +91,8 @@ public class IterativeDataStream<IN> extends
List<String> name = Arrays.asList(new String[] { iterationName });
- if (iterationTail instanceof ConnectedDataStream) {
- for (DataStream<IN> stream : ((ConnectedDataStream<IN>) iterationTail).connectedStreams) {
+ if (iterationTail instanceof MergedDataStream) {
+ for (DataStream<IN> stream : ((MergedDataStream<IN>) iterationTail).mergedStreams) {
String inputID = stream.getId();
jobGraphBuilder.setEdge(inputID, returnStream.getId(),
new ForwardPartitioner<IN>(), 0, name);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
new file mode 100755
index 0000000..6481398
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
@@ -0,0 +1,84 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.partitioner.StreamPartitioner;
+
+/**
+ * The MergedDataStream represents a DataStream which consists of merged outputs
+ * of DataStreams of the same type. Operators applied on this will transform all
+ * the merged outputs jointly.
+ *
+ * @param <OUT>
+ * Type of the output.
+ */
+public class MergedDataStream<OUT> extends DataStream<OUT> {
+
+ protected List<DataStream<OUT>> mergedStreams;
+
+ protected MergedDataStream(StreamExecutionEnvironment environment, String operatorType) {
+ super(environment, operatorType);
+ this.mergedStreams = new ArrayList<DataStream<OUT>>();
+ this.mergedStreams.add(this);
+ }
+
+ protected MergedDataStream(DataStream<OUT> dataStream) {
+ super(dataStream);
+ mergedStreams = new ArrayList<DataStream<OUT>>();
+ if (dataStream instanceof MergedDataStream) {
+ for (DataStream<OUT> stream : ((MergedDataStream<OUT>) dataStream).mergedStreams) {
+ mergedStreams.add(stream);
+ }
+ } else {
+ this.mergedStreams.add(this);
+ }
+
+ }
+
+ // @Override
+ // public IterativeDataStream<OUT> iterate() {
+ // throw new RuntimeException("Cannot iterate connected DataStreams");
+ // }
+
+ protected void addConnection(DataStream<OUT> stream) {
+ mergedStreams.add(stream.copy());
+ }
+
+ @Override
+ protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
+ MergedDataStream<OUT> returnStream = (MergedDataStream<OUT>) this.copy();
+
+ for (DataStream<OUT> stream : returnStream.mergedStreams) {
+ stream.partitioner = partitioner;
+ }
+
+ return returnStream;
+ }
+
+ @Override
+ protected MergedDataStream<OUT> copy() {
+ return new MergedDataStream<OUT>(this);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
new file mode 100755
index 0000000..c64bd33
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
@@ -0,0 +1,70 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import java.util.Iterator;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.MutableTableState;
+
+public class BatchGroupReduceInvokable<IN> extends BatchReduceInvokable<IN, IN> {
+
+ int keyPosition;
+ protected ReduceFunction<IN> reducer;
+ private Iterator<StreamRecord<IN>> iterator;
+ private MutableTableState<Object, IN> values;
+
+ public BatchGroupReduceInvokable(ReduceFunction<IN> reduceFunction, long batchSize,
+ long slideSize, int keyPosition) {
+ super(reduceFunction, batchSize, slideSize);
+ this.keyPosition = keyPosition;
+ this.reducer = reduceFunction;
+ values = new MutableTableState<Object, IN>();
+ }
+
+ @Override
+ protected void reduce() throws Exception {
+ iterator = state.getStreamRecordIterator();
+ while (iterator.hasNext()) {
+ StreamRecord<IN> nextRecord = iterator.next();
+
+ IN nextValue = nextRecord.getObject();
+ Object key = nextRecord.getField(keyPosition);
+
+ IN currentValue = values.get(key);
+ if (currentValue != null) {
+ IN reduced = reducer.reduce(currentValue, nextValue);
+ values.put(key, reduced);
+ collector.collect(reduced);
+ } else {
+ values.put(key, nextValue);
+ collector.collect(nextValue);
+ }
+ }
+ System.out.println(values);
+ values.clear();
+ System.out.println(values);
+
+ }
+
+ private static final long serialVersionUID = 1L;
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index ffd4f1a..3d1d813 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -19,88 +19,56 @@
package org.apache.flink.streaming.api.invokable.operator;
-import java.util.ArrayList;
-
import org.apache.commons.math.util.MathUtils;
import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.state.SlidingWindowState;
public class BatchReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
- private int batchSize;
- private int slideSize;
- private int granularity;
- private boolean emitted;
- private transient SlidingWindowState<IN> state;
+ private long batchSize;
+ int counter = 0;
- public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, int batchSize,
- int slideSize) {
- super(reduceFunction);
+ public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long batchSize,
+ long slideSize) {
+ super(reduceFunction);
this.reducer = reduceFunction;
this.batchSize = batchSize;
this.slideSize = slideSize;
this.granularity = MathUtils.gcd(batchSize, slideSize);
+ this.listSize = (int) granularity;
}
-
- @Override
- protected void immutableInvoke() throws Exception {
- reuse = loadNextRecord();
- ArrayList<IN> list;
-
- while (!state.isFull()) {
- list = new ArrayList<IN>(granularity);
- try {
- state.pushBack(fillArray(list));
- } catch (NullPointerException e) {
- throw new RuntimeException("DataStream length must be greater than batchsize");
- }
- }
-
- boolean go = reduce();
-
- while (go) {
- if (state.isEmittable()) {
- go = reduce();
- } else {
- list = (ArrayList<IN>) state.popFront();
- list.clear();
- state.pushBack(fillArray(list));
- emitted = false;
- go = reuse != null;
- }
- }
- if (!emitted) {
- reduce();
- }
- }
-
- private boolean reduce() throws Exception {
- userIterator = state.getIterator();
- reducer.reduce(userIterable, collector);
- emitted = true;
- return reuse != null;
- }
-
- private ArrayList<IN> fillArray(ArrayList<IN> list) {
- int counter = 0;
- do {
- counter++;
- list.add(reuse.getObject());
- resetReuse();
- } while ((reuse = loadNextRecord()) != null && counter < granularity);
- return list;
+
+ protected BatchReduceInvokable(ReduceFunction<IN> reduceFunction, long batchSize,
+ long slideSize) {
+ super(reduceFunction);
+ this.batchSize = batchSize;
+ this.slideSize = slideSize;
+ this.granularity = MathUtils.gcd(batchSize, slideSize);
+ this.listSize = (int) granularity;
}
@Override
protected void mutableInvoke() throws Exception {
throw new RuntimeException("Reducing mutable sliding batch is not supported.");
}
-
+
@Override
- public void open(Configuration parameters) throws Exception{
+ public void open(Configuration parameters) throws Exception {
super.open(parameters);
this.state = new SlidingWindowState<IN>(batchSize, slideSize, granularity);
}
+ @Override
+ protected boolean batchNotFull() {
+ counter++;
+ if (counter < granularity) {
+ return true;
+ } else {
+ counter = 0;
+ return false;
+ }
+ }
+
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
index 67af978..277078b 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
@@ -19,24 +19,22 @@
package org.apache.flink.streaming.api.invokable.operator;
-import java.util.HashMap;
-import java.util.Map;
-
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
+import org.apache.flink.streaming.state.MutableTableState;
public class GroupReduceInvokable<IN> extends UserTaskInvokable<IN, IN> {
private static final long serialVersionUID = 1L;
private ReduceFunction<IN> reducer;
private int keyPosition;
- private Map<Object, IN> values;
+ private MutableTableState<Object, IN> values;
public GroupReduceInvokable(ReduceFunction<IN> reducer, int keyPosition) {
super(reducer);
this.reducer = reducer;
this.keyPosition = keyPosition;
- values = new HashMap<Object, IN>();
+ values = new MutableTableState<Object, IN>();
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
index de0a18a..f6515dc 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
@@ -19,12 +19,15 @@
package org.apache.flink.streaming.api.invokable.operator;
+import java.util.ArrayList;
import java.util.Iterator;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.GroupReduceFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.SlidingWindowState;
public abstract class StreamReduceInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
@@ -36,6 +39,10 @@ public abstract class StreamReduceInvokable<IN, OUT> extends UserTaskInvokable<I
protected GroupReduceFunction<IN, OUT> reducer;
protected BatchIterator<IN> userIterator;
protected BatchIterable userIterable;
+ protected long slideSize;
+ protected long granularity;
+ protected int listSize;
+ protected transient SlidingWindowState<IN> state;
@Override
public void open(Configuration parameters) throws Exception {
@@ -43,6 +50,45 @@ public abstract class StreamReduceInvokable<IN, OUT> extends UserTaskInvokable<I
super.open(parameters);
}
+ @Override
+ protected void immutableInvoke() throws Exception {
+ if ((reuse = loadNextRecord()) == null) {
+ throw new RuntimeException("DataStream must not be empty");
+ }
+
+ while (reuse != null && !state.isFull()) {
+ collectOneUnit();
+ }
+ reduce();
+
+ while (reuse != null) {
+ for (int i = 0; i < slideSize / granularity; i++) {
+ if (reuse != null) {
+ collectOneUnit();
+ }
+ }
+ reduce();
+ }
+ }
+
+ protected void reduce() throws Exception {
+ userIterator = state.getIterator();
+ reducer.reduce(userIterable, collector);
+ }
+
+ private void collectOneUnit() {
+ ArrayList<StreamRecord<IN>> list;
+ list = new ArrayList<StreamRecord<IN>>(listSize);
+
+ do {
+ list.add(reuse);
+ resetReuse();
+ } while ((reuse = loadNextRecord()) != null && batchNotFull());
+ state.pushBack(list);
+ }
+
+ protected abstract boolean batchNotFull();
+
protected class BatchIterable implements Iterable<IN> {
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
new file mode 100755
index 0000000..065df25
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
@@ -0,0 +1,67 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import java.util.Iterator;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.MutableTableState;
+
+public class WindowGroupReduceInvokable<IN> extends WindowReduceInvokable<IN, IN> {
+
+ int keyPosition;
+ protected ReduceFunction<IN> reducer;
+ private Iterator<StreamRecord<IN>> iterator;
+ private MutableTableState<Object, IN> values;
+
+ public WindowGroupReduceInvokable(ReduceFunction<IN> reduceFunction, long windowSize,
+ long slideInterval, int keyPosition) {
+ super(reduceFunction, windowSize, slideInterval);
+ this.keyPosition = keyPosition;
+ this.reducer = reduceFunction;
+ values = new MutableTableState<Object, IN>();
+ }
+
+ @Override
+ protected void reduce() throws Exception {
+ iterator = state.getStreamRecordIterator();
+ while (iterator.hasNext()) {
+ StreamRecord<IN> nextRecord = iterator.next();
+
+ IN nextValue = nextRecord.getObject();
+ Object key = nextRecord.getField(keyPosition);
+
+ IN currentValue = values.get(key);
+ if (currentValue != null) {
+ IN reduced = reducer.reduce(currentValue, nextValue);
+ values.put(key, reduced);
+ collector.collect(reduced);
+ } else {
+ values.put(key, nextValue);
+ collector.collect(nextValue);
+ }
+ }
+ values.clear();
+ }
+
+ private static final long serialVersionUID = 1L;
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index cbc242c..905db51 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -19,72 +19,57 @@
package org.apache.flink.streaming.api.invokable.operator;
-import java.util.ArrayList;
-
+import org.apache.commons.math.util.MathUtils;
import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.state.SlidingWindowState;
public class WindowReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private long windowSize;
- private long slideInterval;
- private long timeUnitInMillis;
- private transient SlidingWindowState<IN> state;
volatile boolean isRunning;
+ private long startTime;
public WindowReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize,
- long slideInterval, long timeUnitInMillis) {
+ long slideInterval) {
super(reduceFunction);
+ this.reducer = reduceFunction;
this.windowSize = windowSize;
- this.slideInterval = slideInterval;
- this.timeUnitInMillis = timeUnitInMillis;
- }
-
- protected void immutableInvoke() throws Exception {
- if ((reuse = loadNextRecord()) == null) {
- throw new RuntimeException("DataStream must not be empty");
- }
-
- while (reuse != null && !state.isFull()) {
- collectOneTimeUnit();
- }
- reduce();
-
- while (reuse != null) {
- for (int i = 0; i < slideInterval / timeUnitInMillis; i++) {
- collectOneTimeUnit();
- }
- reduce();
- }
+ this.slideSize = slideInterval;
+ this.granularity = MathUtils.gcd(windowSize, slideSize);
+ this.listSize = (int) granularity;
}
-
- private void collectOneTimeUnit() {
- ArrayList<IN> list;
- list = new ArrayList<IN>();
- long startTime = System.currentTimeMillis();
-
- do {
- list.add(reuse.getObject());
- resetReuse();
- } while ((reuse = loadNextRecord()) != null
- && System.currentTimeMillis() - startTime < timeUnitInMillis);
- state.pushBack(list);
+
+ public WindowReduceInvokable(ReduceFunction<IN> reduceFunction, long windowSize,
+ long slideInterval) {
+ super(reduceFunction);
+ this.windowSize = windowSize;
+ this.slideSize = slideInterval;
+ this.granularity = MathUtils.gcd(windowSize, slideSize);
+ this.listSize = (int) granularity;
}
- private boolean reduce() throws Exception {
- userIterator = state.forceGetIterator();
- reducer.reduce(userIterable, collector);
- return reuse != null;
+ @Override
+ protected boolean batchNotFull() {
+ long time = System.currentTimeMillis();
+ if (time - startTime < granularity) {
+ return true;
+ } else {
+ startTime = time;
+ return false;
+ }
}
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
- this.state = new SlidingWindowState<IN>(windowSize, slideInterval, timeUnitInMillis);
+ startTime = System.currentTimeMillis();
+ this.state = new SlidingWindowState<IN>(windowSize, slideSize, granularity);
}
protected void mutableInvoke() throws Exception {
throw new RuntimeException("Reducing mutable sliding window is not supported.");
}
+
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/MutableTableState.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/MutableTableState.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/MutableTableState.java
index 0152361..f5ee31c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/MutableTableState.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/MutableTableState.java
@@ -20,8 +20,6 @@
package org.apache.flink.streaming.state;
import java.io.Serializable;
-import java.util.LinkedHashMap;
-import java.util.Map;
/**
* The most general internal state that stores data in a mutable map.
@@ -29,7 +27,6 @@ import java.util.Map;
public class MutableTableState<K, V> extends TableState<K, V> implements Serializable {
private static final long serialVersionUID = 1L;
- private Map<K, V> state = new LinkedHashMap<K, V>();
@Override
public void put(K key, V value) {
@@ -56,4 +53,9 @@ public class MutableTableState<K, V> extends TableState<K, V> implements Seriali
return new MutableTableStateIterator<K, V>(state.entrySet().iterator());
}
+ @Override
+ public String toString() {
+ return state.toString();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
index a062ba9..f352399 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
@@ -20,9 +20,11 @@
package org.apache.flink.streaming.state;
import java.io.Serializable;
+import java.util.Iterator;
import java.util.List;
import org.apache.commons.collections.buffer.CircularFifoBuffer;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
/**
* The window state for window operator. To be general enough, this class
@@ -49,14 +51,14 @@ public class SlidingWindowState<T> implements Serializable {
this.iterator = new SlidingWindowStateIterator<T>(buffer);
}
- public void pushBack(List<T> array) {
+ public void pushBack(List<StreamRecord<T>> array) {
buffer.add(array);
currentRecordCount += 1;
}
@SuppressWarnings("unchecked")
- public List<T> popFront() {
- List<T> frontRecord = (List<T>) buffer.get();
+ public List<StreamRecord<T>> popFront() {
+ List<StreamRecord<T>> frontRecord = (List<StreamRecord<T>>) buffer.get();
buffer.remove();
return frontRecord;
}
@@ -66,17 +68,13 @@ public class SlidingWindowState<T> implements Serializable {
}
public SlidingWindowStateIterator<T> getIterator() {
- if (isFull()) {
- iterator.reset();
- return iterator;
- } else {
- return null;
- }
+ iterator.reset();
+ return iterator;
}
- public SlidingWindowStateIterator<T> forceGetIterator() {
+ public Iterator<StreamRecord<T>> getStreamRecordIterator() {
iterator.reset();
- return iterator;
+ return iterator.getStreamRecordIterator();
}
public boolean isEmittable() {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
index 6033276..ab0f36e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
@@ -24,15 +24,18 @@ import java.util.Iterator;
import org.apache.commons.collections.buffer.CircularFifoBuffer;
import org.apache.flink.streaming.api.invokable.operator.BatchIterator;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
public class SlidingWindowStateIterator<T> implements BatchIterator<T> {
private CircularFifoBuffer buffer;
- private Iterator<Collection<T>> iterator;
- private Iterator<T> subIterator;
-
+ private Iterator<Collection<StreamRecord<T>>> iterator;
+ private Iterator<StreamRecord<T>> subIterator;
+ private Iterator<StreamRecord<T>> streamRecordIterator;
+
public SlidingWindowStateIterator(CircularFifoBuffer buffer) {
this.buffer = buffer;
+ this.streamRecordIterator = new StreamRecordIterator();
}
public boolean hasNext() {
@@ -40,7 +43,7 @@ public class SlidingWindowStateIterator<T> implements BatchIterator<T> {
}
public T next() {
- T nextElement = subIterator.next();
+ T nextElement = subIterator.next().getObject();
if (!subIterator.hasNext()) {
if (iterator.hasNext()) {
subIterator = iterator.next().iterator();
@@ -60,4 +63,33 @@ public class SlidingWindowStateIterator<T> implements BatchIterator<T> {
iterator = buffer.iterator();
subIterator = iterator.next().iterator();
}
+
+ public Iterator<StreamRecord<T>> getStreamRecordIterator() {
+ return this.streamRecordIterator;
+ }
+
+ private class StreamRecordIterator implements Iterator<StreamRecord<T>> {
+
+ @Override
+ public boolean hasNext() {
+ return SlidingWindowStateIterator.this.hasNext();
+ }
+
+ @Override
+ public StreamRecord<T> next() {
+ StreamRecord<T> nextElement = subIterator.next();
+ if (!subIterator.hasNext()) {
+ if (iterator.hasNext()) {
+ subIterator = iterator.next().iterator();
+ }
+ }
+ return nextElement;
+ }
+
+ @Override
+ public void remove() {
+ SlidingWindowStateIterator.this.remove();
+ }
+
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/TableState.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/TableState.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/TableState.java
index cd3e23a..3e0ed50 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/TableState.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/TableState.java
@@ -29,7 +29,7 @@ import java.util.Map;
@SuppressWarnings("serial")
public class TableState<K, V> implements Serializable {
- private Map<K, V> state=new LinkedHashMap<K, V>();
+ protected Map<K, V> state=new LinkedHashMap<K, V>();
public void put(K key, V value) {
state.put(key, value);
@@ -50,4 +50,8 @@ public class TableState<K, V> implements Serializable {
public TableStateIterator<K, V> getIterator() {
return new TableStateIterator<K, V>(state.entrySet().iterator());
}
+
+ public void clear(){
+ state.clear();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
index 83c98fc..90fac36 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
@@ -25,18 +25,15 @@ import org.apache.flink.streaming.util.LogUtils;
import org.apache.log4j.Level;
import org.junit.Test;
-public class PrintTest {
+public class PrintTest{
-
private static final long MEMORYSIZE = 32;
-
@Test
public void test() throws Exception {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
env.generateSequence(1, 10).print();
env.executeTest(MEMORYSIZE);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
index 020dae4..068efe2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
@@ -80,12 +80,12 @@ public class CoMapTest implements Serializable {
DataStream<Integer> ds1 = env.fromElements(1, 3);
@SuppressWarnings("unchecked")
- DataStream<Integer> ds2 = env.fromElements(2, 4).connectWith(ds1);
+ DataStream<Integer> ds2 = env.fromElements(2, 4).merge(ds1);
DataStream<String> ds3 = env.fromElements("a", "b");
@SuppressWarnings({ "unused", "unchecked" })
- DataStream<Boolean> ds4 = env.fromElements("c").connectWith(ds3).co(ds2).map(new MyCoMap())
+ DataStream<Boolean> ds4 = env.fromElements("c").merge(ds3).connect(ds2).map(new MyCoMap())
.addSink(new EmptySink());
env.executeTest(32);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
index 0c59864..55624d6 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
@@ -318,7 +318,7 @@ public class MapTest {
DataStream<Integer> source1 = env.addSource(new MySource1(), 1);
@SuppressWarnings({ "unused", "unchecked" })
- DataStream<Integer> source2 = env.addSource(new MySource2(), 1).connectWith(source1)
+ DataStream<Integer> source2 = env.addSource(new MySource2(), 1).merge(source1)
.partitionBy(0).map(new MySingleJoinMap()).setParallelism(1)
.addSink(new JoinSink());
@@ -331,7 +331,7 @@ public class MapTest {
@SuppressWarnings({ "unused", "unchecked" })
DataStream<Integer> source5 = env.addSource(new MySource3(), 1)
- .connectWith(source3, source4).partitionBy(0).map(new MyMultipleJoinMap())
+ .merge(source3, source4).partitionBy(0).map(new MyMultipleJoinMap())
.setParallelism(1).addSink(new JoinSink());
env.executeTest(MEMORYSIZE);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
index 15902f4..0c8673b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
@@ -26,6 +26,7 @@ import java.util.Arrays;
import java.util.SortedSet;
import java.util.TreeSet;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.junit.Test;
public class SlidingWindowStateTest {
@@ -34,14 +35,15 @@ public class SlidingWindowStateTest {
private final static int SLIDE_SIZE = 2;
private static final int UNIT = 1;
+ @SuppressWarnings("unchecked")
@Test
public void test() {
SlidingWindowState<Integer> state = new SlidingWindowState<Integer>(SLIDING_BATCH_SIZE,
SLIDE_SIZE, UNIT);
- state.pushBack(Arrays.asList(0));
- state.pushBack(Arrays.asList(1));
+ state.pushBack(Arrays.asList(new StreamRecord<Integer>().setObject(0)));
+ state.pushBack(Arrays.asList(new StreamRecord<Integer>().setObject(1)));
assertEquals(false, state.isFull());
- state.pushBack(Arrays.asList(2));
+ state.pushBack(Arrays.asList(new StreamRecord<Integer>().setObject(2)));
assertTrue(state.isFull());
SlidingWindowStateIterator<Integer> iterator = state.getIterator();
@@ -53,9 +55,9 @@ public class SlidingWindowStateTest {
assertEquals(getExpectedSet(0, 2), actualSet);
actualSet.clear();
- state.pushBack(Arrays.asList(3));
+ state.pushBack(Arrays.asList(new StreamRecord<Integer>().setObject(3)));
assertEquals(false, state.isEmittable());
- state.pushBack(Arrays.asList(4));
+ state.pushBack(Arrays.asList(new StreamRecord<Integer>().setObject(4)));
assertTrue(state.isEmittable());
iterator = state.getIterator();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
index 4f98f5a..352ab70 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
@@ -123,7 +123,7 @@ public class CellInfoLocal {
new QuerySource(), SOURCE_PARALLELISM);
DataStream<Tuple1<String>> stream = env.addSource(new InfoSource(), SOURCE_PARALLELISM)
- .connectWith(querySource).partitionBy(1).flatMap(new CellTask());
+ .merge(querySource).partitionBy(1).flatMap(new CellTask());
stream.print();
env.execute();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
index 08738a2..537047e 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
@@ -46,7 +46,7 @@ public class JoinLocal {
@SuppressWarnings("unused")
DataStream<Tuple3<String, Integer, Integer>> source2 = env
- .addSource(new JoinSourceTwo(), SOURCE_PARALLELISM).connectWith(source1)
+ .addSource(new JoinSourceTwo(), SOURCE_PARALLELISM).merge(source1)
.partitionBy(1).flatMap(new JoinTask()).addSink(new JoinSink());
env.execute();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index 4e9022b..125fae6 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -135,7 +135,7 @@ public class IncrementalLearningSkeleton {
@SuppressWarnings("unchecked")
DataStream<Tuple1<Integer>> prediction = env
- .addSource(new NewDataSource(), SOURCE_PARALLELISM).connectWith(model)
+ .addSource(new NewDataSource(), SOURCE_PARALLELISM).merge(model)
.map(new Predictor());
prediction.print();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
index 078b514..a2d38a5 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
@@ -165,7 +165,7 @@ public class IncrementalOLS {
@SuppressWarnings("unchecked")
DataStream<Tuple1<Double>> prediction = env
- .addSource(new NewDataSource(), SOURCE_PARALLELISM).connectWith(model)
+ .addSource(new NewDataSource(), SOURCE_PARALLELISM).merge(model)
.map(new Predictor());
prediction.print();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
index 83cdd52..2487858 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
@@ -46,7 +46,7 @@ public class WindowJoinLocal {
@SuppressWarnings("unchecked")
DataStream<Tuple3<String, Integer, Integer>> dataStream2 = env
- .addSource(new WindowJoinSourceTwo(), SOURCE_PARALLELISM).connectWith(dataStream1)
+ .addSource(new WindowJoinSourceTwo(), SOURCE_PARALLELISM).merge(dataStream1)
.partitionBy(1).flatMap(new WindowJoinTask());
dataStream2.print();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/696bce02/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
index ba5a6e2..1149a59 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
@@ -36,7 +36,8 @@ public class WordCountLocal {
DataStream<Tuple2<String, Integer>> dataStream = env
.readTextFile("src/test/resources/testdata/hamlet.txt")
.flatMap(new WordCountSplitter())
- .groupReduce(new WordCountCounter(), 0);
+ .groupBy(0)
+ .reduce(new WordCountCounter());
dataStream.print();
[22/51] [abbrv] git commit: [streaming] Added Flume connector and
updated connectors
Posted by se...@apache.org.
[streaming] Added Flume connector and updated connectors
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/a06f9d10
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/a06f9d10
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/a06f9d10
Branch: refs/heads/master
Commit: a06f9d10e6ea9f91dc6723af483ba9328d4e1621
Parents: 7ca778b
Author: jfeher <fe...@gmail.com>
Authored: Tue Jul 29 13:51:34 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:21:13 2014 +0200
----------------------------------------------------------------------
.../flink-streaming-connectors/pom.xml | 11 +-
.../streaming/connectors/flume/FlumeSink.java | 129 +++++++++++++++++++
.../streaming/connectors/flume/FlumeSource.java | 114 ++++++++++++++++
.../connectors/flume/FlumeTopology.java | 88 +++++++++++++
.../connectors/kafka/KafkaProducer.java | 75 -----------
.../streaming/connectors/kafka/KafkaSink.java | 24 +++-
.../streaming/connectors/kafka/KafkaSource.java | 34 ++---
.../connectors/kafka/KafkaTopology.java | 38 +++---
.../streaming/connectors/rabbitmq/RMQSink.java | 74 +++++------
.../connectors/rabbitmq/RMQSource.java | 52 ++++----
.../connectors/rabbitmq/RMQTopology.java | 29 ++---
11 files changed, 472 insertions(+), 196 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
index ac15d55..8db610f 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
@@ -48,14 +48,19 @@ under the License.
<version>0.8.0</version>
</dependency>
-
- <dependency>
+ <dependency>
<groupId>com.rabbitmq</groupId>
<artifactId>amqp-client</artifactId>
<version>3.3.1</version>
</dependency>
- <dependency>
+ <dependency>
+ <groupId>org.apache.flume</groupId>
+ <artifactId>flume-ng-core</artifactId>
+ <version>1.5.0</version>
+ </dependency>
+
+ <dependency>
<groupId>com.twitter</groupId>
<artifactId>hbc-core</artifactId>
<version>2.2.0</version>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
new file mode 100644
index 0000000..27ce546
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
@@ -0,0 +1,129 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.connectors.flume;
+
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.FlumeException;
+import org.apache.flume.api.RpcClient;
+import org.apache.flume.api.RpcClientFactory;
+import org.apache.flume.event.EventBuilder;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+
+public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
+ private static final long serialVersionUID = 1L;
+
+ private transient FlinkRpcClientFacade client;
+ boolean initDone = false;
+ String host;
+ int port;
+ private boolean sendAndClose = false;
+ private boolean closeWithoutSend = false;
+
+ public FlumeSink(String host, int port) {
+ this.host = host;
+ this.port = port;
+ }
+
+ @Override
+ public void invoke(IN tuple) {
+
+ if (!initDone) {
+ client = new FlinkRpcClientFacade();
+ client.init(host, port);
+ }
+
+ byte[] data = serialize(tuple);
+ if(!closeWithoutSend){
+ client.sendDataToFlume(data);
+ }
+ if (sendAndClose) {
+ client.close();
+ }
+
+ }
+
+ public abstract byte[] serialize(IN tuple);
+
+ private class FlinkRpcClientFacade {
+ private RpcClient client;
+ private String hostname;
+ private int port;
+
+ public void init(String hostname, int port) {
+ // Setup the RPC connection
+ this.hostname = hostname;
+ this.port = port;
+ int initCounter = 0;
+ while(true){
+ if(initCounter >= 90){
+ System.exit(1);
+ }
+ try{
+ this.client = RpcClientFactory.getDefaultInstance(hostname, port);
+ }
+ catch(FlumeException e){
+ // Wait one second if the connection failed before the next try
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e1) {
+ e1.printStackTrace();
+ }
+ }
+ if(client != null){
+ break;
+ }
+ initCounter++;
+ }
+ initDone = true;
+ }
+
+ public void sendDataToFlume(byte[] data) {
+ Event event = EventBuilder.withBody(data);
+
+ try {
+ client.append(event);
+
+ } catch (EventDeliveryException e) {
+ // clean up and recreate the client
+ client.close();
+ client = null;
+ client = RpcClientFactory.getDefaultInstance(hostname, port);
+ }
+ }
+
+ public void close() {
+ client.close();
+ }
+
+ }
+
+ public void sendAndClose() {
+ sendAndClose = true;
+ }
+
+ public void closeWithoutSend(){
+ client.close();
+ closeWithoutSend=true;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
new file mode 100644
index 0000000..966c881
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
@@ -0,0 +1,114 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.connectors.flume;
+
+import java.util.List;
+
+import org.apache.flume.Context;
+import org.apache.flume.channel.ChannelProcessor;
+import org.apache.flume.source.AvroSource;
+import org.apache.flume.source.avro.AvroFlumeEvent;
+import org.apache.flume.source.avro.Status;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.util.Collector;
+
+public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
+ private static final long serialVersionUID = 1L;
+
+ String host;
+ String port;
+
+ FlumeSource(String host, int port){
+ this.host = host;
+ this.port = Integer.toString(port);
+ }
+
+ public class MyAvroSource extends AvroSource {
+ Collector<IN> collector;
+
+ @Override
+ public Status append(AvroFlumeEvent avroEvent) {
+ collect(avroEvent);
+ return Status.OK;
+ }
+
+ @Override
+ public Status appendBatch(List<AvroFlumeEvent> events) {
+ for (AvroFlumeEvent avroEvent : events) {
+ collect(avroEvent);
+ }
+
+ return Status.OK;
+ }
+
+ private void collect(AvroFlumeEvent avroEvent){
+ byte[] b = avroEvent.getBody().array();
+ IN tuple = FlumeSource.this.deserialize(b);
+ if(!closeWithoutSend){
+ collector.collect(tuple);
+ }
+ if(sendAndClose){
+ sendDone = true;
+ }
+ }
+
+ }
+
+ MyAvroSource avroSource;
+ private volatile boolean closeWithoutSend = false;
+ private boolean sendAndClose = false;
+ private volatile boolean sendDone = false;
+ public abstract IN deserialize(byte[] msg);
+
+ public void configureAvroSource(Collector<IN> collector){
+
+ avroSource = new MyAvroSource();
+ avroSource.collector=collector;
+ Context context = new Context();
+ context.put("port", port);
+ context.put("bind", host);
+ avroSource.configure(context);
+ ChannelProcessor cp = new ChannelProcessor(null);
+ avroSource.setChannelProcessor(cp);
+ }
+
+ @Override
+ public void invoke(Collector<IN> collector) throws Exception {
+ configureAvroSource(collector);
+ avroSource.start();
+ while(true){
+ if(closeWithoutSend || sendDone){
+ break;
+ }
+ }
+ avroSource.stop();
+ }
+
+ public void sendAndClose(){
+ sendAndClose = true;
+ }
+
+ public void closeWithoutSend(){
+ closeWithoutSend = true;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
new file mode 100644
index 0000000..e08837a
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
@@ -0,0 +1,88 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.connectors.flume;
+
+import org.apache.commons.lang.SerializationUtils;
+
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+
+public class FlumeTopology {
+
+ public static class MyFlumeSink extends FlumeSink<Tuple1<String>> {
+ private static final long serialVersionUID = 1L;
+
+ public MyFlumeSink(String host, int port) {
+ super(host, port);
+ }
+
+ @Override
+ public byte[] serialize(Tuple1<String> tuple) {
+ if (tuple.f0.equals("q")) {
+ try {
+ sendAndClose();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ return SerializationUtils.serialize((String) tuple.getField(0));
+ }
+
+ }
+
+ public static class MyFlumeSource extends FlumeSource<Tuple1<String>> {
+ private static final long serialVersionUID = 1L;
+
+ MyFlumeSource(String host, int port) {
+ super(host, port);
+ }
+
+ @Override
+ public Tuple1<String> deserialize(byte[] msg) {
+ String s = (String) SerializationUtils.deserialize(msg);
+ Tuple1<String> out = new Tuple1<String>();
+ out.f0 = s;
+ if (s.equals("q")) {
+ closeWithoutSend();
+ }
+ return out;
+ }
+
+ }
+
+ public static void main(String[] args) throws Exception {
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment
+ .createLocalEnvironment(1);
+
+ @SuppressWarnings("unused")
+ DataStream<Tuple1<String>> dataStream1 = env
+ .addSource(new MyFlumeSource("localhost", 41414))
+ .print();
+
+ @SuppressWarnings("unused")
+ DataStream<Tuple1<String>> dataStream2 = env
+ .fromElements("one", "two", "three", "four", "five", "q")
+ .addSink(new MyFlumeSink("localhost", 42424));
+
+ env.execute();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducer.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducer.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducer.java
deleted file mode 100644
index 00a8964..0000000
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducer.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import java.io.BufferedReader;
-import java.io.FileReader;
-import java.util.Properties;
-
-import kafka.producer.KeyedMessage;
-import kafka.producer.ProducerConfig;
-
-/**
- * This is a simple kafka producer that reads local file from disk and produces
- * line streams. To use the producer, a zookeeper server and a kafka server
- * should be in service. Run the following script to start a zookeeper server:
- * bin/zookeeper-server-start.sh config/zookeeper.properties Run the following
- * script to start a kafka server: bin/kafka-server-start.sh
- * config/server.properties Run the following script to start the producer: java
- * -cp kafka-0.8/libs/*:yourJarFile.jar
- * org.apache.flink.streaming.kafka.KafkaProducer yourTopicID kafkaServerIp As
- * an example: java -cp kafka-0.8/libs/*:flink-streaming.jar
- * org.apache.flink.streaming.kafka.KafkaProducer test localhost:9092
- */
-public class KafkaProducer {
- static kafka.javaapi.producer.Producer<Integer, String> producer;
- static Properties props = new Properties();
-
- public static void ProducerPrepare(String brokerAddr) {
- props.put("serializer.class", "kafka.serializer.StringEncoder");
- props.put("metadata.broker.list", brokerAddr);
-
- producer = new kafka.javaapi.producer.Producer<Integer, String>(
- new ProducerConfig(props));
- }
-
- public static void main(String[] args) throws Exception {
- if (args.length == 3) {
- String infilename = args[0];
- String topicId = args[1];
- String brokerAddr = args[2];
- ProducerPrepare(brokerAddr);
- BufferedReader reader = new BufferedReader(new FileReader(
- infilename));
- while (true) {
- String line = reader.readLine();
- if (line == null) {
- reader.close();
- reader = new BufferedReader(new FileReader(infilename));
- continue;
- }
- producer.send(new KeyedMessage<Integer, String>(topicId, line));
- }
- } else {
- System.out.println("please set filename!");
- System.exit(-1);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
index eeac961..e97cdd0 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
@@ -35,7 +35,8 @@ public abstract class KafkaSink<IN extends Tuple, OUT> extends SinkFunction<IN>
static Properties props;
private String topicId;
private String brokerAddr;
- private boolean close = false;
+ private boolean sendAndClose = false;
+ private boolean closeWithoutSend = false;
private boolean initDone = false;
public KafkaSink(String topicId, String brokerAddr) {
@@ -63,18 +64,27 @@ public abstract class KafkaSink<IN extends Tuple, OUT> extends SinkFunction<IN>
}
OUT out = serialize(tuple);
- KeyedMessage<Integer, OUT> data = new KeyedMessage<Integer, OUT>(topicId, out);
- producer.send(data);
- if (close) {
- producer.close();
+ KeyedMessage<Integer, OUT> data = new KeyedMessage<Integer, OUT>(
+ topicId, out);
+
+ if (!closeWithoutSend) {
+ producer.send(data);
}
+ if (sendAndClose) {
+ producer.close();
+ }
}
public abstract OUT serialize(IN tuple);
- public void close() {
- close = true;
+ public void closeWithoutSend() {
+ producer.close();
+ closeWithoutSend = true;
+ }
+
+ public void sendAndClose() {
+ sendAndClose = true;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
index d34b6c3..32b11d9 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
@@ -24,21 +24,19 @@ import java.util.List;
import java.util.Map;
import java.util.Properties;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-
import kafka.consumer.ConsumerConfig;
import kafka.consumer.ConsumerIterator;
import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
-
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
/**
- * Source for reading messages from a Kafka queue.
- * The source currently only support string messages.
+ * Source for reading messages from a Kafka queue. The source currently only
+ * support string messages.
*/
-public abstract class KafkaSource<IN extends Tuple, OUT> extends SourceFunction<IN>{
+public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
private static final long serialVersionUID = 1L;
private final String zkQuorum;
@@ -46,7 +44,8 @@ public abstract class KafkaSource<IN extends Tuple, OUT> extends SourceFunction<
private final String topicId;
private final int numThreads;
private ConsumerConnector consumer;
- private boolean close = false;
+ private boolean closeWithoutSend = false;
+ private boolean sendAndClose = false;
IN outTuple;
@@ -81,20 +80,25 @@ public abstract class KafkaSource<IN extends Tuple, OUT> extends SourceFunction<
ConsumerIterator<byte[], byte[]> it = stream.iterator();
while (it.hasNext()) {
- IN out=deserialize(it.next().message());
- if(!close){
- collector.collect(out);
+ IN out = deserialize(it.next().message());
+ if (closeWithoutSend) {
+ break;
}
- else {
+ collector.collect(out);
+ if (sendAndClose) {
break;
}
}
consumer.shutdown();
}
-
+
public abstract IN deserialize(byte[] msg);
-
- public void close(){
- close=true;
+
+ public void closeWithoutSend() {
+ closeWithoutSend = true;
+ }
+
+ public void sendAndClose() {
+ sendAndClose = true;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
index 1a77aee..e111feb 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
@@ -19,10 +19,10 @@
package org.apache.flink.streaming.connectors.kafka;
+import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
public class KafkaTopology {
@@ -31,8 +31,8 @@ public class KafkaTopology {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Collector<Tuple1<String>> collector) throws Exception {
- // TODO Auto-generated method stub
+ public void invoke(Collector<Tuple1<String>> collector)
+ throws Exception {
for (int i = 0; i < 10; i++) {
collector.collect(new Tuple1<String>(Integer.toString(i)));
}
@@ -41,39 +41,37 @@ public class KafkaTopology {
}
}
- public static final class MyKafkaSource extends KafkaSource<Tuple1<String>, String> {
+ public static final class MyKafkaSource extends KafkaSource<Tuple1<String>> {
private static final long serialVersionUID = 1L;
- public MyKafkaSource(String zkQuorum, String groupId, String topicId, int numThreads) {
+ public MyKafkaSource(String zkQuorum, String groupId, String topicId,
+ int numThreads) {
super(zkQuorum, groupId, topicId, numThreads);
- // TODO Auto-generated constructor stub
}
@Override
public Tuple1<String> deserialize(byte[] msg) {
- // TODO Auto-generated method stub
String s = new String(msg);
if (s.equals("q")) {
- close();
+ closeWithoutSend();
}
return new Tuple1<String>(s);
}
}
- public static final class MyKafkaSink extends KafkaSink<Tuple1<String>, String> {
+ public static final class MyKafkaSink extends
+ KafkaSink<Tuple1<String>, String> {
private static final long serialVersionUID = 1L;
public MyKafkaSink(String topicId, String brokerAddr) {
super(topicId, brokerAddr);
- // TODO Auto-generated constructor stub
}
@Override
public String serialize(Tuple1<String> tuple) {
- // TODO Auto-generated method stub
if (tuple.f0.equals("q")) {
- close();
+ sendAndClose();
}
return tuple.f0;
}
@@ -84,15 +82,19 @@ public class KafkaTopology {
public static void main(String[] args) {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+ StreamExecutionEnvironment env = StreamExecutionEnvironment
+ .createLocalEnvironment(1);
@SuppressWarnings("unused")
- DataStream<Tuple1<String>> stream1 = env.addSource(
- new MyKafkaSource("localhost:2181", "group", "test", 1), SOURCE_PARALELISM).print();
+ DataStream<Tuple1<String>> stream1 = env
+ .addSource(new MyKafkaSource("localhost:2181", "group", "test", 1), SOURCE_PARALELISM)
+ .print();
@SuppressWarnings("unused")
- DataStream<Tuple1<String>> stream2 = env.addSource(new MySource()).addSink(
- new MyKafkaSink("test", "localhost:9092"));
+ DataStream<Tuple1<String>> stream2 = env
+ .addSource(new MySource())
+ .addSink(new MyKafkaSink("test", "localhost:9092"));
+
env.execute();
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index e6e8de5..5440ea7 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -21,47 +21,39 @@ package org.apache.flink.streaming.connectors.rabbitmq;
import java.io.IOException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-
import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
import com.rabbitmq.client.ConnectionFactory;
-/**
- * Source for sending messages to a RabbitMQ queue. The source currently only
- * support string messages. Other types will be added soon.
- *
- */
-public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
- private static final Log LOG = LogFactory.getLog(RMQSink.class);
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
- private boolean close = false;
+ private boolean sendAndClose = false;
+ private boolean closeWithoutSend = false;
- private String queueName;
- private String hostName;
+ private String QUEUE_NAME;
+ private String HOST_NAME;
private transient ConnectionFactory factory;
private transient Connection connection;
private transient Channel channel;
private boolean initDone = false;
- public RMQSink(String hostName, String queueName) {
- this.hostName = hostName;
- this.queueName = queueName;
+ public RMQSink(String HOST_NAME, String QUEUE_NAME) {
+ this.HOST_NAME = HOST_NAME;
+ this.QUEUE_NAME = QUEUE_NAME;
}
public void initializeConnection() {
factory = new ConnectionFactory();
- factory.setHost(hostName);
+ factory.setHost(HOST_NAME);
try {
connection = factory.newConnection();
channel = connection.createChannel();
+
} catch (IOException e) {
- new RuntimeException("Cannot create RMQ connection with " + queueName + " at "
- + hostName, e);
+ e.printStackTrace();
}
initDone = true;
@@ -74,31 +66,39 @@ public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
}
try {
- channel.queueDeclare(queueName, false, false, false, null);
+ channel.queueDeclare(QUEUE_NAME, false, false, false, null);
byte[] msg = serialize(tuple);
- channel.basicPublish("", queueName, null, msg);
- } catch (IOException e) {
- if (LOG.isErrorEnabled()) {
- LOG.error("Cannot send RMQ message to " + queueName + " at " + hostName);
+ if (!closeWithoutSend) {
+ channel.basicPublish("", QUEUE_NAME, null, msg);
}
+ } catch (IOException e) {
+ e.printStackTrace();
}
- if (close) {
- try {
- channel.close();
- connection.close();
- } catch (IOException e) {
- if (LOG.isWarnEnabled()) {
- LOG.warn("Cannot close RMQ connection: " + queueName + " at " + hostName);
- }
- }
+ if (sendAndClose) {
+ closeChannel();
}
}
public abstract byte[] serialize(Tuple t);
- public void close() {
- close = true;
+ private void closeChannel() {
+ try {
+ channel.close();
+ connection.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ }
+
+ public void closeWithoutSend() {
+ closeChannel();
+ closeWithoutSend = true;
+ }
+
+ public void sendAndClose() {
+ sendAndClose = true;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index 4b197e3..345d884 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -21,31 +21,24 @@ package org.apache.flink.streaming.connectors.rabbitmq;
import java.io.IOException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.util.Collector;
-
import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
import com.rabbitmq.client.ConnectionFactory;
+import com.rabbitmq.client.ConsumerCancelledException;
import com.rabbitmq.client.QueueingConsumer;
+import com.rabbitmq.client.ShutdownSignalException;
-/**
- * Source for reading messages from a RabbitMQ queue. The source currently only
- * support string messages. Other types will be added soon.
- *
- */
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.util.Collector;
public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
- private static final Log LOG = LogFactory.getLog(RMQSource.class);
-
private static final long serialVersionUID = 1L;
private final String QUEUE_NAME;
private final String HOST_NAME;
- private boolean close = false;
+ private boolean closeWithoutSend = false;
+ private boolean sendAndClose = false;
private transient ConnectionFactory factory;
private transient Connection connection;
@@ -70,8 +63,6 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
consumer = new QueueingConsumer(channel);
channel.basicConsume(QUEUE_NAME, true, consumer);
} catch (IOException e) {
- new RuntimeException("Cannot create RMQ connection with " + QUEUE_NAME + " at "
- + HOST_NAME, e);
}
}
@@ -79,20 +70,29 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
public void invoke(Collector<IN> collector) throws Exception {
initializeConnection();
- while (!close) {
+ while (true) {
try {
delivery = consumer.nextDelivery();
- } catch (Exception e) {
- if (LOG.isErrorEnabled()) {
- LOG.error("Cannot receive RMQ message " + QUEUE_NAME + " at " + HOST_NAME);
- }
+ } catch (ShutdownSignalException e) {
+ e.printStackTrace();
+ break;
+ } catch (ConsumerCancelledException e) {
+ e.printStackTrace();
+ break;
+ } catch (InterruptedException e) {
+ e.printStackTrace();
}
outTuple = deserialize(delivery.getBody());
- if (!close){
+ if (closeWithoutSend) {
+ break;
+ } else {
collector.collect(outTuple);
}
+ if (sendAndClose) {
+ break;
+ }
}
try {
@@ -105,8 +105,12 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
public abstract IN deserialize(byte[] t);
- public void close() {
- close = true;
+ public void closeWithoutSend() {
+ closeWithoutSend = true;
+ }
+
+ public void sendAndClose() {
+ sendAndClose = true;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a06f9d10/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
index 4b7c5dc..f84505b 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
@@ -19,10 +19,8 @@
package org.apache.flink.streaming.connectors.rabbitmq;
-import java.util.HashSet;
-import java.util.Set;
-
import org.apache.commons.lang.SerializationUtils;
+
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
@@ -33,16 +31,14 @@ public class RMQTopology {
public static final class MyRMQSink extends RMQSink<Tuple1<String>> {
public MyRMQSink(String HOST_NAME, String QUEUE_NAME) {
super(HOST_NAME, QUEUE_NAME);
- // TODO Auto-generated constructor stub
}
private static final long serialVersionUID = 1L;
@Override
public byte[] serialize(Tuple t) {
- // TODO Auto-generated method stub
if (t.getField(0).equals("q")) {
- close();
+ sendAndClose();
}
return SerializationUtils.serialize((String) t.getField(0));
}
@@ -53,7 +49,6 @@ public class RMQTopology {
public MyRMQSource(String HOST_NAME, String QUEUE_NAME) {
super(HOST_NAME, QUEUE_NAME);
- // TODO Auto-generated constructor stub
}
private static final long serialVersionUID = 1L;
@@ -64,28 +59,28 @@ public class RMQTopology {
Tuple1<String> out = new Tuple1<String>();
out.f0 = s;
if (s.equals("q")) {
- close();
+ closeWithoutSend();
}
return out;
}
}
- @SuppressWarnings("unused")
- private static Set<String> result = new HashSet<String>();
-
public static void main(String[] args) throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+ StreamExecutionEnvironment env = StreamExecutionEnvironment
+ .createLocalEnvironment(1);
@SuppressWarnings("unused")
- DataStream<Tuple1<String>> dataStream1 = env.addSource(
- new MyRMQSource("localhost", "hello")).print();
+ DataStream<Tuple1<String>> dataStream1 = env
+ .addSource(new MyRMQSource("localhost", "hello"))
+ .print();
@SuppressWarnings("unused")
- DataStream<Tuple1<String>> dataStream2 = env.fromElements("one", "two", "three", "four",
- "five", "q").addSink(new MyRMQSink("localhost", "hello"));
+ DataStream<Tuple1<String>> dataStream2 = env
+ .fromElements("one", "two", "three", "four", "five", "q")
+ .addSink(new MyRMQSink("localhost", "hello"));
env.execute();
}
-}
\ No newline at end of file
+}
[15/51] [abbrv] git commit: [streaming] Operator invokable refactor
Posted by se...@apache.org.
[streaming] Operator invokable refactor
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/be459aec
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/be459aec
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/be459aec
Branch: refs/heads/master
Commit: be459aece580ea0f4ea9e028cf29b15d7d7f33f4
Parents: 1fccb10
Author: gyfora <gy...@gmail.com>
Authored: Wed Jul 23 14:53:27 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:19:18 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 31 +++-
.../streaming/api/invokable/SinkInvokable.java | 21 +--
.../api/invokable/StreamRecordInvokable.java | 23 ++-
.../operator/BatchReduceInvokable.java | 156 ++++---------------
.../api/invokable/operator/FilterInvokable.java | 26 ++--
.../invokable/operator/FlatMapInvokable.java | 22 +--
.../api/invokable/operator/MapInvokable.java | 21 +--
.../operator/StreamReduceInvokable.java | 32 ++++
.../operator/WindowReduceInvokable.java | 125 +++++++++++++++
9 files changed, 276 insertions(+), 181 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/be459aec/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index d32aa18..27e4d89 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -36,6 +36,7 @@ import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
+import org.apache.flink.streaming.api.invokable.operator.WindowReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
import org.apache.flink.streaming.partitioner.DistributePartitioner;
@@ -292,7 +293,7 @@ public class DataStream<T extends Tuple> {
public DataStream<T> forward() {
return setConnectionType(new ForwardPartitioner<T>());
}
-
+
/**
* Sets the partitioning of the {@link DataStream} so that the output tuples
* are distributed evenly to the next component.
@@ -332,6 +333,27 @@ public class DataStream<T extends Tuple> {
}
/**
+ * Applies a CoMap transformation on two separate {@link DataStream}s. The
+ * transformation calls a {@link CoMapFunction#map1(Tuple)} for each element
+ * of the first DataStream (on which .coMapWith was called) and
+ * {@link CoMapFunction#map2(Tuple)} for each element of the second
+ * DataStream. Each CoMapFunction call returns exactly one element.
+ *
+ * @param coMapper
+ * The CoMapFunction used to jointly transform the two input
+ * DataStreams
+ * @param otherStream
+ * The DataStream that will be transformed with
+ * {@link CoMapFunction#map2(Tuple)}
+ * @return The transformed DataStream
+ */
+ public <T2 extends Tuple, R extends Tuple> DataStream<R> coMapWith(
+ CoMapFunction<T, T2, R> coMapper, DataStream<T2> otherStream) {
+ return environment.addCoFunction("coMap", new DataStream<T>(this), new DataStream<T2>(
+ otherStream), coMapper, new CoMapInvokable<T, T2, R>(coMapper));
+ }
+
+ /**
* Applies a FlatMap transformation on a {@link DataStream}. The
* transformation calls a FlatMapFunction for each element of the
* DataStream. Each FlatMapFunction call can return any number of elements
@@ -387,11 +409,6 @@ public class DataStream<T extends Tuple> {
new BatchReduceInvokable<T, R>(reducer, batchSize));
}
- public <T2 extends Tuple, R extends Tuple> DataStream<R> coMapWith(CoMapFunction<T, T2, R> coMapper, DataStream<T2> otherStream) {
- return environment.addCoFunction("coMap", new DataStream<T>(this), new DataStream<T2>(otherStream), coMapper, new CoMapInvokable<T, T2, R>(coMapper));
- }
-
-
/**
* Applies a reduce transformation on preset "time" chunks of the
* DataStream. The transformation calls a {@link GroupReduceFunction} on
@@ -411,7 +428,7 @@ public class DataStream<T extends Tuple> {
public <R extends Tuple> StreamOperator<T, R> windowReduce(GroupReduceFunction<T, R> reducer,
long windowSize) {
return environment.addFunction("batchReduce", new DataStream<T>(this), reducer,
- new BatchReduceInvokable<T, R>(reducer, windowSize));
+ new WindowReduceInvokable<T, R>(reducer, windowSize));
}
/**
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/be459aec/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index 3c14490..81cfa81 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -32,16 +32,17 @@ public class SinkInvokable<IN extends Tuple> extends StreamRecordInvokable<IN, I
}
@Override
- public void invoke() throws Exception {
- if (this.isMutable) {
- while (recordIterator.next(reuse) != null) {
- sinkFunction.invoke((IN) reuse.getTuple());
- }
- } else {
- while (recordIterator.next(reuse) != null) {
- sinkFunction.invoke((IN) reuse.getTuple());
- resetReuse();
- }
+ protected void immutableInvoke() throws Exception {
+ while (recordIterator.next(reuse) != null) {
+ sinkFunction.invoke((IN) reuse.getTuple());
+ resetReuse();
+ }
+ }
+
+ @Override
+ protected void mutableInvoke() throws Exception {
+ while (recordIterator.next(reuse) != null) {
+ sinkFunction.invoke((IN) reuse.getTuple());
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/be459aec/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
index 903372b..6beec27 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
@@ -19,6 +19,8 @@
package org.apache.flink.streaming.api.invokable;
+import java.io.IOException;
+
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
@@ -49,6 +51,25 @@ public abstract class StreamRecordInvokable<IN extends Tuple, OUT extends Tuple>
protected void resetReuse() {
this.reuse = serializer.createInstance();
}
+
+ protected StreamRecord<IN> loadNextRecord() {
+ try {
+ reuse = recordIterator.next(reuse);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ return reuse;
+ }
+
+ protected abstract void immutableInvoke() throws Exception;
- public abstract void invoke() throws Exception;
+ protected abstract void mutableInvoke() throws Exception;
+
+ public void invoke() throws Exception {
+ if (this.isMutable) {
+ mutableInvoke();
+ } else {
+ immutableInvoke();
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/be459aec/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
old mode 100644
new mode 100755
index 7684f70..4aa540c
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -19,109 +19,53 @@
package org.apache.flink.streaming.api.invokable.operator;
-import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.java.functions.GroupReduceFunction;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
- UserTaskInvokable<IN, OUT> {
+ StreamReduceInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
private int batchSize;
- private long windowSize;
- volatile boolean isRunning;
- boolean window;
-
- private GroupReduceFunction<IN, OUT> reducer;
public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, int batchSize) {
this.reducer = reduceFunction;
this.batchSize = batchSize;
}
- public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize) {
- this.reducer = reduceFunction;
- this.windowSize = windowSize;
- this.window = true;
- }
-
- private StreamRecord<IN> loadNextRecord() {
- try {
- reuse = recordIterator.next(reuse);
- } catch (IOException e) {
- e.printStackTrace();
- }
- return reuse;
- }
-
@Override
- public void invoke() throws Exception {
- if (this.isMutable) {
- mutableInvoke();
- } else {
- immutableInvoke();
- }
- }
-
- private void immutableInvoke() throws Exception {
+ protected void immutableInvoke() throws Exception {
List<IN> tupleBatch = new ArrayList<IN>();
boolean batchStart;
-
- if (window) {
- long startTime = System.currentTimeMillis();
- while (loadNextRecord() != null) {
- batchStart = true;
- do {
- if (batchStart) {
- batchStart = false;
- } else {
- reuse = loadNextRecord();
- if (reuse == null) {
- break;
- }
+ int counter = 0;
+
+ while (loadNextRecord() != null) {
+ batchStart = true;
+ do {
+ if (batchStart) {
+ batchStart = false;
+ } else {
+ reuse = loadNextRecord();
+ if (reuse == null) {
+ break;
}
- tupleBatch.add(reuse.getTuple());
- resetReuse();
- } while (System.currentTimeMillis() - startTime < windowSize);
- reducer.reduce(tupleBatch.iterator(), collector);
- tupleBatch.clear();
- startTime = System.currentTimeMillis();
- }
- } else {
- int counter = 0;
- while (loadNextRecord() != null) {
- batchStart = true;
- do {
- if (batchStart) {
- batchStart = false;
- } else {
- reuse = loadNextRecord();
- if (reuse == null) {
- break;
- }
- }
- counter++;
- tupleBatch.add(reuse.getTuple());
- resetReuse();
- } while (counter < batchSize);
- reducer.reduce(tupleBatch.iterator(), collector);
- tupleBatch.clear();
- counter = 0;
- }
+ }
+ counter++;
+ tupleBatch.add(reuse.getTuple());
+ resetReuse();
+ } while (counter < batchSize);
+ reducer.reduce(tupleBatch.iterator(), collector);
+ tupleBatch.clear();
+ counter = 0;
}
+
}
- private void mutableInvoke() throws Exception {
- BatchIterator<IN> userIterator;
- if (window) {
- userIterator = new WindowIterator();
- } else {
- userIterator = new CounterIterator();
- }
+ @Override
+ protected void mutableInvoke() throws Exception {
+ BatchIterator<IN> userIterator = new CounterIterator();
do {
if (userIterator.hasNext()) {
@@ -131,7 +75,7 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
} while (reuse != null);
}
- public class CounterIterator implements BatchIterator<IN> {
+ private class CounterIterator implements BatchIterator<IN> {
private int counter;
private boolean loadedNext;
@@ -179,52 +123,4 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
}
- public class WindowIterator implements BatchIterator<IN> {
-
- volatile boolean iterate;
- private boolean loadedNext;
- private long startTime;
-
- public WindowIterator() {
- startTime = System.currentTimeMillis();
- }
-
- @Override
- public boolean hasNext() {
- if (System.currentTimeMillis() - startTime > windowSize) {
- return false;
- } else if (!loadedNext) {
- loadNextRecord();
- loadedNext = true;
- }
- return (reuse != null);
- }
-
- @Override
- public IN next() {
- if (hasNext()) {
- loadedNext = false;
- return reuse.getTuple();
- } else {
- loadedNext = false;
- return reuse.getTuple();
- }
- }
-
- public void reset() {
- while (System.currentTimeMillis() - startTime < windowSize) {
- loadNextRecord();
- }
- loadNextRecord();
- loadedNext = true;
- startTime = System.currentTimeMillis();
- }
-
- @Override
- public void remove() {
-
- }
-
- }
-
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/be459aec/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
index ac79764..edeb79a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
@@ -34,21 +34,21 @@ public class FilterInvokable<IN extends Tuple> extends UserTaskInvokable<IN, IN>
}
@Override
- public void invoke() throws Exception {
- if (this.isMutable) {
- while (recordIterator.next(reuse) != null) {
- if (filterFunction.filter(reuse.getTuple())) {
- collector.collect(reuse.getTuple());
- }
- }
- } else {
- while (recordIterator.next(reuse) != null) {
- if (filterFunction.filter(reuse.getTuple())) {
- collector.collect(reuse.getTuple());
- }
- resetReuse();
+ protected void immutableInvoke() throws Exception {
+ while (recordIterator.next(reuse) != null) {
+ if (filterFunction.filter(reuse.getTuple())) {
+ collector.collect(reuse.getTuple());
}
+ resetReuse();
}
+ }
+ @Override
+ protected void mutableInvoke() throws Exception {
+ while (recordIterator.next(reuse) != null) {
+ if (filterFunction.filter(reuse.getTuple())) {
+ collector.collect(reuse.getTuple());
+ }
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/be459aec/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
index 33bda80..279b160 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
@@ -33,16 +33,18 @@ public class FlatMapInvokable<IN extends Tuple, OUT extends Tuple> extends
this.flatMapper = flatMapper;
}
- public void invoke() throws Exception {
- if (this.isMutable) {
- while (recordIterator.next(reuse) != null) {
- flatMapper.flatMap(reuse.getTuple(), collector);
- }
- } else {
- while (recordIterator.next(reuse) != null) {
- flatMapper.flatMap(reuse.getTuple(), collector);
- resetReuse();
- }
+ @Override
+ protected void immutableInvoke() throws Exception {
+ while (recordIterator.next(reuse) != null) {
+ flatMapper.flatMap(reuse.getTuple(), collector);
+ resetReuse();
+ }
+ }
+
+ @Override
+ protected void mutableInvoke() throws Exception {
+ while (recordIterator.next(reuse) != null) {
+ flatMapper.flatMap(reuse.getTuple(), collector);
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/be459aec/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
index ff29d15..3c56b6f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
@@ -33,16 +33,17 @@ public class MapInvokable<IN extends Tuple, OUT extends Tuple> extends UserTaskI
}
@Override
- public void invoke() throws Exception {
- if (this.isMutable) {
- while (recordIterator.next(reuse) != null) {
- collector.collect(mapper.map(reuse.getTuple()));
- }
- } else {
- while (recordIterator.next(reuse) != null) {
- collector.collect(mapper.map(reuse.getTuple()));
- resetReuse();
- }
+ protected void immutableInvoke() throws Exception {
+ while (recordIterator.next(reuse) != null) {
+ collector.collect(mapper.map(reuse.getTuple()));
+ resetReuse();
+ }
+ }
+
+ @Override
+ protected void mutableInvoke() throws Exception {
+ while (recordIterator.next(reuse) != null) {
+ collector.collect(mapper.map(reuse.getTuple()));
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/be459aec/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
new file mode 100644
index 0000000..e881d57
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
@@ -0,0 +1,32 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import org.apache.flink.api.java.functions.GroupReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
+
+public abstract class StreamReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
+ UserTaskInvokable<IN, OUT> {
+ private static final long serialVersionUID = 1L;
+ protected GroupReduceFunction<IN, OUT> reducer;
+
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/be459aec/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
new file mode 100755
index 0000000..67c15dc
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -0,0 +1,125 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.functions.GroupReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+
+public class WindowReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
+ StreamReduceInvokable<IN, OUT> {
+ private static final long serialVersionUID = 1L;
+ private long windowSize;
+ volatile boolean isRunning;
+ boolean window;
+
+ public WindowReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize) {
+ this.reducer = reduceFunction;
+ this.windowSize = windowSize;
+ this.window = true;
+ }
+
+ protected void immutableInvoke() throws Exception {
+ List<IN> tupleBatch = new ArrayList<IN>();
+ boolean batchStart;
+
+ long startTime = System.currentTimeMillis();
+ while (loadNextRecord() != null) {
+ batchStart = true;
+ do {
+ if (batchStart) {
+ batchStart = false;
+ } else {
+ reuse = loadNextRecord();
+ if (reuse == null) {
+ break;
+ }
+ }
+ tupleBatch.add(reuse.getTuple());
+ resetReuse();
+ } while (System.currentTimeMillis() - startTime < windowSize);
+ reducer.reduce(tupleBatch.iterator(), collector);
+ tupleBatch.clear();
+ startTime = System.currentTimeMillis();
+ }
+
+ }
+
+ protected void mutableInvoke() throws Exception {
+ BatchIterator<IN> userIterator = new WindowIterator();
+
+ do {
+ if (userIterator.hasNext()) {
+ reducer.reduce(userIterator, collector);
+ userIterator.reset();
+ }
+ } while (reuse != null);
+ }
+
+ private class WindowIterator implements BatchIterator<IN> {
+
+ private boolean loadedNext;
+ private long startTime;
+
+ public WindowIterator() {
+ startTime = System.currentTimeMillis();
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (System.currentTimeMillis() - startTime > windowSize) {
+ return false;
+ } else if (!loadedNext) {
+ loadNextRecord();
+ loadedNext = true;
+ }
+ return (reuse != null);
+ }
+
+ @Override
+ public IN next() {
+ if (hasNext()) {
+ loadedNext = false;
+ return reuse.getTuple();
+ } else {
+ loadedNext = false;
+ return reuse.getTuple();
+ }
+ }
+
+ public void reset() {
+ while (System.currentTimeMillis() - startTime < windowSize) {
+ loadNextRecord();
+ }
+ loadNextRecord();
+ loadedNext = true;
+ startTime = System.currentTimeMillis();
+ }
+
+ @Override
+ public void remove() {
+
+ }
+
+ }
+
+}
\ No newline at end of file
[07/51] [abbrv] git commit: [streaming] Serialization rework to reuse
objects
Posted by se...@apache.org.
[streaming] Serialization rework to reuse objects
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/0c1ef6d2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/0c1ef6d2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/0c1ef6d2
Branch: refs/heads/master
Commit: 0c1ef6d2dcb4b0cb9effc602411ec53fd9bb9ada
Parents: f54d7b9
Author: gyfora <gy...@gmail.com>
Authored: Mon Jul 21 09:28:53 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:14:13 2014 +0200
----------------------------------------------------------------------
.../api/collector/DirectedStreamCollector.java | 9 +-
.../api/collector/StreamCollector.java | 24 +--
.../api/invokable/DefaultSinkInvokable.java | 7 +-
.../api/invokable/DefaultTaskInvokable.java | 6 +-
.../streaming/api/invokable/SinkInvokable.java | 13 +-
.../api/invokable/StreamRecordInvokable.java | 20 ++-
.../operator/BatchReduceInvokable.java | 54 ++++--
.../api/invokable/operator/FilterInvokable.java | 17 +-
.../invokable/operator/FlatMapInvokable.java | 12 +-
.../api/invokable/operator/MapInvokable.java | 11 +-
.../AbstractStreamComponent.java | 93 +++++-----
.../streamcomponent/StreamIterationSink.java | 32 ++--
.../streamcomponent/StreamIterationSource.java | 16 +-
.../api/streamcomponent/StreamRecordReader.java | 169 -------------------
.../api/streamcomponent/StreamSink.java | 15 +-
.../api/streamcomponent/StreamSource.java | 9 +-
.../api/streamcomponent/StreamTask.java | 41 ++---
.../UnionStreamRecordReader.java | 91 ----------
.../api/streamrecord/StreamRecord.java | 36 +---
.../streamrecord/StreamRecordSerializer.java | 119 +++++++++++++
.../partitioner/BroadcastPartitioner.java | 4 +-
.../partitioner/FieldsPartitioner.java | 10 +-
.../partitioner/ForwardPartitioner.java | 4 +-
.../partitioner/GlobalPartitioner.java | 4 +-
.../partitioner/ShufflePartitioner.java | 6 +-
.../partitioner/StreamPartitioner.java | 3 +-
.../apache/flink/streaming/api/PrintTest.java | 19 +--
.../api/invokable/operator/BatchReduceTest.java | 5 +-
.../api/streamcomponent/MockRecordWriter.java | 0
29 files changed, 357 insertions(+), 492 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
index b059041..f968b83 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
@@ -51,7 +51,8 @@ public class DirectedStreamCollector<T extends Tuple> extends StreamCollector<T>
* @param outputSelector
* User defined {@link OutputSelector}
*/
- public DirectedStreamCollector(int channelID, SerializationDelegate<T> serializationDelegate,
+ public DirectedStreamCollector(int channelID,
+ SerializationDelegate<StreamRecord<T>> serializationDelegate,
OutputSelector<T> outputSelector) {
super(channelID, serializationDelegate);
this.outputSelector = outputSelector;
@@ -81,10 +82,12 @@ public class DirectedStreamCollector<T extends Tuple> extends StreamCollector<T>
private void emit(StreamRecord<T> streamRecord) {
Collection<String> outputNames = outputSelector.getOutputs(streamRecord.getTuple());
streamRecord.setId(channelID);
+ serializationDelegate.setInstance(streamRecord);
for (String outputName : outputNames) {
try {
- for (RecordWriter<StreamRecord<T>> output : outputMap.get(outputName)) {
- output.emit(streamRecord);
+ for (RecordWriter<SerializationDelegate<StreamRecord<T>>> output : outputMap
+ .get(outputName)) {
+ output.emit(serializationDelegate);
}
} catch (Exception e) {
if (log.isErrorEnabled()) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
index 47ec8f0..544a695 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
@@ -47,8 +47,9 @@ public class StreamCollector<T extends Tuple> implements Collector<T> {
protected StreamRecord<T> streamRecord;
protected int channelID;
- private List<RecordWriter<StreamRecord<T>>> outputs;
- protected Map<String, List<RecordWriter<StreamRecord<T>>>> outputMap;
+ private List<RecordWriter<SerializationDelegate<StreamRecord<T>>>> outputs;
+ protected Map<String, List<RecordWriter<SerializationDelegate<StreamRecord<T>>>>> outputMap;
+ protected SerializationDelegate<StreamRecord<T>> serializationDelegate;
/**
* Creates a new StreamCollector
@@ -58,13 +59,13 @@ public class StreamCollector<T extends Tuple> implements Collector<T> {
* @param serializationDelegate
* Serialization delegate used for tuple serialization
*/
- public StreamCollector(int channelID, SerializationDelegate<T> serializationDelegate) {
+ public StreamCollector(int channelID, SerializationDelegate<StreamRecord<T>> serializationDelegate) {
+ this.serializationDelegate = serializationDelegate;
this.streamRecord = new StreamRecord<T>();
- this.streamRecord.setSeralizationDelegate(serializationDelegate);
this.channelID = channelID;
- this.outputs = new ArrayList<RecordWriter<StreamRecord<T>>>();
- this.outputMap = new HashMap<String, List<RecordWriter<StreamRecord<T>>>>();
+ this.outputs = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<T>>>>();
+ this.outputMap = new HashMap<String, List<RecordWriter<SerializationDelegate<StreamRecord<T>>>>>();
}
/**
@@ -75,13 +76,15 @@ public class StreamCollector<T extends Tuple> implements Collector<T> {
* @param outputName
* User defined name of the output.
*/
- public void addOutput(RecordWriter<StreamRecord<T>> output, String outputName) {
+ public void addOutput(RecordWriter<SerializationDelegate<StreamRecord<T>>> output,
+ String outputName) {
outputs.add(output);
if (outputName != null) {
if (outputMap.containsKey(outputName)) {
outputMap.get(outputName).add(output);
} else {
- outputMap.put(outputName,new ArrayList<RecordWriter<StreamRecord<T>>>());
+ outputMap.put(outputName,
+ new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<T>>>>());
outputMap.get(outputName).add(output);
}
@@ -109,9 +112,10 @@ public class StreamCollector<T extends Tuple> implements Collector<T> {
*/
private void emit(StreamRecord<T> streamRecord) {
streamRecord.setId(channelID);
- for (RecordWriter<StreamRecord<T>> output : outputs) {
+ serializationDelegate.setInstance(streamRecord);
+ for (RecordWriter<SerializationDelegate<StreamRecord<T>>> output : outputs) {
try {
- output.emit(streamRecord);
+ output.emit(serializationDelegate);
} catch (Exception e) {
if (LOG.isErrorEnabled()) {
LOG.error(String.format("Emit failed due to: %s",
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSinkInvokable.java
index 91c6959..09e766b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultSinkInvokable.java
@@ -19,17 +19,12 @@
package org.apache.flink.streaming.api.invokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.util.Collector;
public class DefaultSinkInvokable<T extends Tuple> extends UserSinkInvokable<T> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(StreamRecord<T> record, Collector<T> collector) throws Exception {
- String value = (String) record.getTuple().getField(0);
- System.out.println(value);
+ public void invoke() throws Exception {
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultTaskInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultTaskInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultTaskInvokable.java
index abf392c..c86ffcb 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultTaskInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/DefaultTaskInvokable.java
@@ -19,17 +19,13 @@
package org.apache.flink.streaming.api.invokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.util.Collector;
public class DefaultTaskInvokable<T extends Tuple> extends UserTaskInvokable<T, T> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(StreamRecord<T> record, Collector<T> collector) throws Exception {
- collector.collect(record.getTuple());
+ public void invoke() throws Exception {
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index cb16307..c6e4e36 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -19,10 +19,8 @@
package org.apache.flink.streaming.api.invokable;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.util.Collector;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
public class SinkInvokable<IN extends Tuple> extends UserSinkInvokable<IN> {
private static final long serialVersionUID = 1L;
@@ -32,10 +30,11 @@ public class SinkInvokable<IN extends Tuple> extends UserSinkInvokable<IN> {
public SinkInvokable(SinkFunction<IN> sinkFunction) {
this.sinkFunction = sinkFunction;
}
-
+
@Override
- public void invoke(StreamRecord<IN> record, Collector<IN> collector) throws Exception {
- IN tuple = record.getTuple();
- sinkFunction.invoke(tuple);
+ public void invoke() throws Exception {
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ sinkFunction.invoke((IN) reuse.getTuple());
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
index 229d409..98eb679 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
@@ -19,16 +19,28 @@
package org.apache.flink.streaming.api.invokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
public abstract class StreamRecordInvokable<IN extends Tuple, OUT extends Tuple> extends
StreamComponentInvokable {
private static final long serialVersionUID = 1L;
- public abstract void invoke(StreamRecord<IN> record, Collector<OUT> collector)
- throws Exception;
+ protected Collector<OUT> collector;
+ protected MutableObjectIterator<StreamRecord<IN>> recordIterator;
+ protected StreamRecord<IN> reuse;
+
+ public void initialize(Collector<OUT> collector,
+ MutableObjectIterator<StreamRecord<IN>> recordIterator,
+ StreamRecordSerializer<IN> serializer) {
+ this.collector = collector;
+ this.recordIterator = recordIterator;
+ this.reuse = serializer.createInstance();
+ }
+
+ public abstract void invoke() throws Exception;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 0b009e9..8e35af2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -19,19 +19,16 @@
package org.apache.flink.streaming.api.invokable.operator;
-import java.util.ArrayList;
-import java.util.List;
+import java.io.IOException;
+import java.util.Iterator;
import org.apache.flink.api.java.functions.GroupReduceFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
- private List<IN> tupleBatch;
private int counter;
private int batchSize;
@@ -39,20 +36,53 @@ public class BatchReduceInvokable<IN extends Tuple, OUT extends Tuple> extends
public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, int batchSize) {
this.reducer = reduceFunction;
- this.tupleBatch = new ArrayList<IN>();
this.counter = 0;
this.batchSize = batchSize;
}
@Override
- public void invoke(StreamRecord<IN> record, Collector<OUT> collector) throws Exception {
+ public void invoke() throws Exception {
+ MyIterator it = new MyIterator();
+ do {
+ reducer.reduce(it, collector);
+ it.reset();
+ } while (reuse != null);
+ }
+
+ public class MyIterator implements Iterator<IN> {
+
+ public MyIterator() {
+ reset();
+ }
+
+ @Override
+ public boolean hasNext() {
+
+ if (counter >= batchSize) {
+ return false;
+ } else {
+ try {
+ reuse = recordIterator.next(reuse);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ return (reuse != null);
+ }
+ }
+
+ @Override
+ public IN next() {
+ counter++;
+ return reuse.getTuple();
+ }
- tupleBatch.add(record.getTuple());
- counter++;
- if (counter >= batchSize) {
+ public void reset() {
counter = 0;
- reducer.reduce(tupleBatch.iterator(), collector);
- tupleBatch.clear();
+ }
+
+ @Override
+ public void remove() {
+
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
index 0d135b5..1dba6b8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
@@ -19,14 +19,11 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.api.java.functions.FilterFunction;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.util.Collector;
+import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-public class FilterInvokable<IN extends Tuple> extends
- UserTaskInvokable<IN, IN> {
+public class FilterInvokable<IN extends Tuple> extends UserTaskInvokable<IN, IN> {
private static final long serialVersionUID = 1L;
@@ -37,11 +34,11 @@ public class FilterInvokable<IN extends Tuple> extends
}
@Override
- public void invoke(StreamRecord<IN> record, Collector<IN> collector)
- throws Exception {
- IN tuple = record.getTuple();
- if (filterFunction.filter(tuple)) {
- collector.collect(tuple);
+ public void invoke() throws Exception {
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ if (filterFunction.filter(reuse.getTuple())) {
+ collector.collect(reuse.getTuple());
+ }
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
index 5696b40..36ab1bf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
@@ -19,11 +19,9 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.api.java.functions.FlatMapFunction;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.util.Collector;
+import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class FlatMapInvokable<IN extends Tuple, OUT extends Tuple> extends
UserTaskInvokable<IN, OUT> {
@@ -35,9 +33,9 @@ public class FlatMapInvokable<IN extends Tuple, OUT extends Tuple> extends
this.flatMapper = flatMapper;
}
- @Override
- public void invoke(StreamRecord<IN> record, Collector<OUT> collector) throws Exception {
- IN tuple = record.getTuple();
- flatMapper.flatMap(tuple, collector);
+ public void invoke() throws Exception {
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ flatMapper.flatMap(reuse.getTuple(), collector);
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
index 378c898..59b9f8f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
@@ -19,11 +19,9 @@
package org.apache.flink.streaming.api.invokable.operator;
-import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.util.Collector;
+import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
public class MapInvokable<IN extends Tuple, OUT extends Tuple> extends UserTaskInvokable<IN, OUT> {
private static final long serialVersionUID = 1L;
@@ -35,8 +33,9 @@ public class MapInvokable<IN extends Tuple, OUT extends Tuple> extends UserTaskI
}
@Override
- public void invoke(StreamRecord<IN> record, Collector<OUT> collector) throws Exception {
- IN tuple = record.getTuple();
- collector.collect(mapper.map(tuple));
+ public void invoke() throws Exception {
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ collector.collect(mapper.map(reuse.getTuple()));
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index 1132276..207d269 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -28,6 +28,7 @@ import org.apache.commons.lang.SerializationUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.FilterFunction;
import org.apache.flink.api.java.functions.FlatMapFunction;
import org.apache.flink.api.java.functions.GroupReduceFunction;
@@ -35,12 +36,14 @@ import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.io.network.api.AbstractRecordReader;
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.MutableRecordReader;
+import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.util.ReaderIterator;
import org.apache.flink.runtime.plugable.DeserializationDelegate;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.collector.DirectedStreamCollector;
@@ -48,24 +51,25 @@ import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.collector.StreamCollector;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
-import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.partitioner.ShufflePartitioner;
import org.apache.flink.streaming.partitioner.StreamPartitioner;
import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tuple> extends
AbstractInvokable {
private static final Log LOG = LogFactory.getLog(AbstractStreamComponent.class);
protected TupleTypeInfo<IN> inTupleTypeInfo = null;
- protected TupleSerializer<IN> inTupleSerializer = null;
- protected DeserializationDelegate<IN> inDeserializationDelegate = null;
+ protected StreamRecordSerializer<IN> inTupleSerializer = null;
+ protected DeserializationDelegate<StreamRecord<IN>> inDeserializationDelegate = null;
protected TupleTypeInfo<OUT> outTupleTypeInfo = null;
- protected TupleSerializer<OUT> outTupleSerializer = null;
- protected SerializationDelegate<OUT> outSerializationDelegate = null;
+ protected StreamRecordSerializer<OUT> outTupleSerializer = null;
+ protected SerializationDelegate<StreamRecord<OUT>> outSerializationDelegate = null;
protected Configuration configuration;
protected StreamCollector<OUT> collector;
@@ -126,16 +130,18 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
} else if (operatorName.equals("elements")) {
outTupleTypeInfo = new TupleTypeInfo<OUT>(TypeExtractor.getForObject(function));
- outTupleSerializer = outTupleTypeInfo.createSerializer();
- outSerializationDelegate = new SerializationDelegate<OUT>(outTupleSerializer);
+ outTupleSerializer = new StreamRecordSerializer<OUT>(outTupleTypeInfo.createSerializer());
+ outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(
+ outTupleSerializer);
} else {
throw new Exception("Wrong operator name!");
}
} catch (Exception e) {
throw new StreamComponentException(e);
-// throw new StreamComponentException("Nonsupported object (named " + operatorName
-// + ") passed as operator");
+ // throw new StreamComponentException("Nonsupported object (named "
+ // + operatorName
+ // + ") passed as operator");
}
}
@@ -149,8 +155,8 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
0, null, null);
- inTupleSerializer = inTupleTypeInfo.createSerializer();
- inDeserializationDelegate = new DeserializationDelegate<IN>(inTupleSerializer);
+ inTupleSerializer = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
+ inDeserializationDelegate = new DeserializationDelegate<StreamRecord<IN>>(inTupleSerializer);
}
@SuppressWarnings({ "rawtypes", "unchecked" })
@@ -158,8 +164,8 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
outTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
typeParameter, null, null);
- outTupleSerializer = outTupleTypeInfo.createSerializer();
- outSerializationDelegate = new SerializationDelegate<OUT>(outTupleSerializer);
+ outTupleSerializer = new StreamRecordSerializer(outTupleTypeInfo.createSerializer());
+ outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(outTupleSerializer);
}
@SuppressWarnings("unchecked")
@@ -167,32 +173,42 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
if (outSerializationDelegate != null) {
inTupleTypeInfo = (TupleTypeInfo<IN>) outTupleTypeInfo;
- inTupleSerializer = inTupleTypeInfo.createSerializer();
- inDeserializationDelegate = new DeserializationDelegate<IN>(inTupleSerializer);
+ inTupleSerializer = new StreamRecordSerializer<IN>(inTupleTypeInfo.createSerializer());
+ inDeserializationDelegate = new DeserializationDelegate<StreamRecord<IN>>(inTupleSerializer);
}
}
@SuppressWarnings("unchecked")
- protected AbstractRecordReader getConfigInputs() throws StreamComponentException {
+ protected MutableReader<IOReadableWritable> getConfigInputs() throws StreamComponentException {
int numberOfInputs = configuration.getInteger("numberOfInputs", 0);
if (numberOfInputs < 2) {
- return new StreamRecordReader<IN>(this, StreamRecord.class, inDeserializationDelegate,
- inTupleSerializer);
+ return new MutableRecordReader<IOReadableWritable>(this);
} else {
- MutableRecordReader<StreamRecord<IN>>[] recordReaders = (MutableRecordReader<StreamRecord<IN>>[]) new MutableRecordReader<?>[numberOfInputs];
+ MutableRecordReader<IOReadableWritable>[] recordReaders = (MutableRecordReader<IOReadableWritable>[]) new MutableRecordReader<?>[numberOfInputs];
for (int i = 0; i < numberOfInputs; i++) {
- recordReaders[i] = new MutableRecordReader<StreamRecord<IN>>(this);
+ recordReaders[i] = new MutableRecordReader<IOReadableWritable>(this);
}
- return new UnionStreamRecordReader<IN>(recordReaders, StreamRecord.class,
- inDeserializationDelegate, inTupleSerializer);
+ return new MutableUnionRecordReader<IOReadableWritable>(recordReaders);
}
}
- protected void setConfigOutputs(List<RecordWriter<StreamRecord<OUT>>> outputs) {
+ protected MutableObjectIterator<StreamRecord<IN>> createInputIterator(MutableReader<?> inputReader,
+ TypeSerializer<?> serializer) {
+
+ // generic data type serialization
+ @SuppressWarnings("unchecked")
+ MutableReader<DeserializationDelegate<?>> reader = (MutableReader<DeserializationDelegate<?>>) inputReader;
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ final MutableObjectIterator<StreamRecord<IN>> iter = new ReaderIterator(reader, serializer);
+ return iter;
+
+ }
+
+ protected void setConfigOutputs(List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
int numberOfOutputs = configuration.getInteger("numberOfOutputs", 0);
@@ -201,7 +217,8 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
}
}
- private void setPartitioner(int outputNumber, List<RecordWriter<StreamRecord<OUT>>> outputs) {
+ private void setPartitioner(int outputNumber,
+ List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
byte[] serializedPartitioner = configuration.getBytes("partitionerObject_" + outputNumber,
SerializationUtils.serialize((new ShufflePartitioner<OUT>())));
@@ -210,8 +227,8 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
try {
outputPartitioner = deserializeObject(serializedPartitioner);
- RecordWriter<StreamRecord<OUT>> output = new RecordWriter<StreamRecord<OUT>>(this,
- outputPartitioner);
+ RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output = new RecordWriter<SerializationDelegate<StreamRecord<OUT>>>(
+ this, outputPartitioner);
outputs.add(output);
String outputName = configuration.getString("outputName_" + outputNumber, null);
@@ -230,26 +247,6 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
}
}
- @SuppressWarnings("unchecked")
- protected void invokeRecords(StreamRecordInvokable<IN, OUT> userFunction,
- AbstractRecordReader inputs) throws Exception {
- if (inputs instanceof UnionStreamRecordReader) {
- UnionStreamRecordReader<IN> recordReader = (UnionStreamRecordReader<IN>) inputs;
- while (recordReader.hasNext()) {
- StreamRecord<IN> record = recordReader.next();
- userFunction.invoke(record, collector);
- }
-
- } else if (inputs instanceof StreamRecordReader) {
- StreamRecordReader<IN> recordReader = (StreamRecordReader<IN>) inputs;
-
- while (recordReader.hasNext()) {
- StreamRecord<IN> record = recordReader.next();
- userFunction.invoke(record, collector);
- }
- }
- }
-
/**
* Reads and creates a StreamComponent from the config.
*
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
index c2e214d..b9d44c3 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
@@ -25,15 +25,18 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.io.network.api.AbstractRecordReader;
+import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.util.MutableObjectIterator;
import org.apache.flink.util.StringUtils;
public class StreamIterationSink<IN extends Tuple> extends AbstractStreamComponent<IN, IN> {
private static final Log LOG = LogFactory.getLog(StreamIterationSink.class);
- private AbstractRecordReader inputs;
+ @SuppressWarnings("rawtypes")
+ private MutableReader inputs;
+ MutableObjectIterator<StreamRecord<IN>> inputIter;
private String iterationId;
@SuppressWarnings("rawtypes")
private BlockingQueue<StreamRecord> dataChannel;
@@ -49,6 +52,7 @@ public class StreamIterationSink<IN extends Tuple> extends AbstractStreamCompone
setSerializers();
setSinkSerializer();
inputs = getConfigInputs();
+ inputIter = createInputIterator(inputs, inTupleSerializer);
iterationId = configuration.getString("iteration-id", "iteration-0");
dataChannel = BlockingQueueBroker.instance().get(iterationId);
} catch (Exception e) {
@@ -63,29 +67,19 @@ public class StreamIterationSink<IN extends Tuple> extends AbstractStreamCompone
LOG.debug("SINK " + name + " invoked");
}
- forwardRecords(inputs);
+ forwardRecords();
if (LOG.isDebugEnabled()) {
LOG.debug("SINK " + name + " invoke finished");
}
}
- @SuppressWarnings("unchecked")
- protected void forwardRecords(AbstractRecordReader inputs) throws Exception {
- if (inputs instanceof UnionStreamRecordReader) {
- UnionStreamRecordReader<IN> recordReader = (UnionStreamRecordReader<IN>) inputs;
- while (recordReader.hasNext()) {
- StreamRecord<IN> record = recordReader.next();
- pushToQueue(record);
- }
-
- } else if (inputs instanceof StreamRecordReader) {
- StreamRecordReader<IN> recordReader = (StreamRecordReader<IN>) inputs;
-
- while (recordReader.hasNext()) {
- StreamRecord<IN> record = recordReader.next();
- pushToQueue(record);
- }
+ protected void forwardRecords() throws Exception {
+ StreamRecord<IN> reuse = inTupleSerializer.createInstance().setId(0);
+ while ((reuse = inputIter.next(reuse)) != null) {
+ pushToQueue(reuse);
+ // TODO: Fix object reuse for iteration
+ reuse = inTupleSerializer.createInstance().setId(0);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
index 7f4da1f..9f58842 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
@@ -29,13 +29,14 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
public class StreamIterationSource<OUT extends Tuple> extends AbstractStreamComponent<Tuple, OUT> {
private static final Log LOG = LogFactory.getLog(StreamIterationSource.class);
- private List<RecordWriter<StreamRecord<OUT>>> outputs;
+ private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private static int numSources;
private int[] numberOfOutputChannels;
private String iterationId;
@@ -45,7 +46,7 @@ public class StreamIterationSource<OUT extends Tuple> extends AbstractStreamComp
@SuppressWarnings("rawtypes")
public StreamIterationSource() {
- outputs = new LinkedList<RecordWriter<StreamRecord<OUT>>>();
+ outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
numSources = newComponent();
instanceID = numSources;
dataChannel = new ArrayBlockingQueue<StreamRecord>(1);
@@ -81,7 +82,7 @@ public class StreamIterationSource<OUT extends Tuple> extends AbstractStreamComp
LOG.debug("SOURCE " + name + " invoked with instance id " + instanceID);
}
- for (RecordWriter<StreamRecord<OUT>> output : outputs) {
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
output.initializeSerializers();
}
@@ -91,14 +92,13 @@ public class StreamIterationSource<OUT extends Tuple> extends AbstractStreamComp
if (nextRecord == null) {
break;
}
- nextRecord.setSeralizationDelegate(this.outSerializationDelegate);
- for (RecordWriter<StreamRecord<OUT>> output : outputs) {
- output.emit(nextRecord);
- // output.flush();
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
+ outSerializationDelegate.setInstance(nextRecord);
+ output.emit(outSerializationDelegate);
}
}
- for (RecordWriter<StreamRecord<OUT>> output : outputs) {
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
output.flush();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordReader.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordReader.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordReader.java
deleted file mode 100755
index d1cc1e0..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordReader.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.streamcomponent;
-
-import java.io.IOException;
-
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.runtime.io.network.api.AbstractSingleGateRecordReader;
-import org.apache.flink.runtime.io.network.api.Reader;
-import org.apache.flink.runtime.io.network.gates.InputChannelResult;
-
-/**
- * A record writer connects an input gate to an application. It allows the
- * application query for incoming records and read them from input gate.
- *
- */
-public class StreamRecordReader<T extends Tuple> extends AbstractSingleGateRecordReader<StreamRecord<T>> implements
- Reader<StreamRecord<T>> {
-
- @SuppressWarnings("rawtypes")
- private final Class<? extends StreamRecord> recordType;
- private DeserializationDelegate<T> deserializationDelegate;
- private TupleSerializer<T> tupleSerializer;
- /**
- * Stores the last read record.
- */
- private StreamRecord<T> lookahead;
-
- /**
- * Stores if more no more records will be received from the assigned input
- * gate.
- */
- private boolean noMoreRecordsWillFollow;
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * Constructs a new record reader and registers a new input gate with the
- * application's environment.
- *
- * @param taskBase
- * The application that instantiated the record reader.
- * @param recordType
- * The class of records that can be read from the record reader.
- * @param deserializationDelegate
- * deserializationDelegate
- * @param tupleSerializer
- * tupleSerializer
- */
- @SuppressWarnings("rawtypes")
- public StreamRecordReader(AbstractInvokable taskBase, Class<? extends StreamRecord> recordType,
- DeserializationDelegate<T> deserializationDelegate,
- TupleSerializer<T> tupleSerializer) {
- 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.
- *
- * @return <code>true</code>it at least one more record can be read from the
- * associated input gate, otherwise <code>false</code>
- */
- @Override
- public boolean hasNext() throws IOException, InterruptedException {
- if (this.lookahead != null) {
- return true;
- } else {
- if (this.noMoreRecordsWillFollow) {
- return false;
- }
-
- StreamRecord<T> 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()) {
- return false;
- } else {
- break; // fall through and wait for next record/event
- }
-
- case TASK_EVENT:
- handleEvent(this.inputGate.getCurrentEvent());
- break;
-
- case END_OF_STREAM:
- this.noMoreRecordsWillFollow = true;
- return false;
-
- default:
- ; // fall through the loop
- }
- }
- }
- }
-
- /**
- * Reads the current record from the associated input gate.
- *
- * @return the current record from the associated input gate.
- * @throws IOException
- * thrown if any error occurs while reading the record from the
- * input gate
- */
- @Override
- public StreamRecord<T> next() throws IOException, InterruptedException {
- if (hasNext()) {
- StreamRecord<T> tmp = this.lookahead;
- this.lookahead = null;
- return tmp;
- } else {
- return null;
- }
- }
-
- @Override
- public boolean isInputClosed() {
- return this.noMoreRecordsWillFollow;
- }
-
- @SuppressWarnings("unchecked")
- private StreamRecord<T> instantiateRecordType() {
- try {
- return this.recordType.newInstance();
- } catch (InstantiationException e) {
- throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName()
- + "'.", e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName()
- + "'.", e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
index f27282a..508a10e9 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
@@ -21,18 +21,21 @@ package org.apache.flink.streaming.api.streamcomponent;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.streaming.api.invokable.DefaultSinkInvokable;
import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
import org.apache.flink.streaming.api.invokable.UserSinkInvokable;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.io.network.api.AbstractRecordReader;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.util.MutableObjectIterator;
public class StreamSink<IN extends Tuple> extends AbstractStreamComponent<IN, IN> {
private static final Log LOG = LogFactory.getLog(StreamSink.class);
- private AbstractRecordReader inputs;
+ @SuppressWarnings("rawtypes")
+ private MutableReader inputs;
+ private MutableObjectIterator<StreamRecord<IN>> inputIter;
private StreamRecordInvokable<IN, IN> userFunction;
public StreamSink() {
@@ -47,6 +50,7 @@ public class StreamSink<IN extends Tuple> extends AbstractStreamComponent<IN, IN
setSerializers();
setSinkSerializer();
inputs = getConfigInputs();
+ inputIter = createInputIterator(inputs, inTupleSerializer);
} catch (Exception e) {
throw new StreamComponentException("Cannot register inputs for "
+ getClass().getSimpleName(), e);
@@ -61,6 +65,7 @@ public class StreamSink<IN extends Tuple> extends AbstractStreamComponent<IN, IN
Class<? extends UserSinkInvokable> userFunctionClass = configuration.getClass(
"userfunction", DefaultSinkInvokable.class, UserSinkInvokable.class);
userFunction = (UserSinkInvokable<IN>) getInvokable(userFunctionClass);
+ userFunction.initialize(collector, inputIter, inTupleSerializer);
}
@Override
@@ -69,7 +74,7 @@ public class StreamSink<IN extends Tuple> extends AbstractStreamComponent<IN, IN
LOG.debug("SINK " + name + " invoked");
}
- invokeRecords(userFunction, inputs);
+ userFunction.invoke();
if (LOG.isDebugEnabled()) {
LOG.debug("SINK " + name + " invoke finished");
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
index 3936ce0..4ac7100 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
@@ -26,6 +26,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.invokable.DefaultSourceInvokable;
import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -34,14 +35,14 @@ public class StreamSource<OUT extends Tuple> extends AbstractStreamComponent<Tup
private static final Log LOG = LogFactory.getLog(StreamSource.class);
- private List<RecordWriter<StreamRecord<OUT>>> outputs;
+ private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private UserSourceInvokable<OUT> userFunction;
private static int numSources;
private int[] numberOfOutputChannels;
public StreamSource() {
- outputs = new LinkedList<RecordWriter<StreamRecord<OUT>>>();
+ outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
userFunction = null;
numSources = newComponent();
instanceID = numSources;
@@ -83,7 +84,7 @@ public class StreamSource<OUT extends Tuple> extends AbstractStreamComponent<Tup
LOG.debug("SOURCE " + name + " invoked with instance id " + instanceID);
}
- for (RecordWriter<StreamRecord<OUT>> output : outputs) {
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
output.initializeSerializers();
}
@@ -93,7 +94,7 @@ public class StreamSource<OUT extends Tuple> extends AbstractStreamComponent<Tup
LOG.debug("SOURCE " + name + " invoke finished with instance id " + instanceID);
}
- for (RecordWriter<StreamRecord<OUT>> output : outputs) {
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
output.flush();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
index 98d6d97..44f93b3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
@@ -25,26 +25,31 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.io.network.api.AbstractRecordReader;
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.invokable.DefaultTaskInvokable;
import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.util.MutableObjectIterator;
-public class StreamTask<IN extends Tuple, OUT extends Tuple> extends AbstractStreamComponent<IN, OUT> {
+public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
+ AbstractStreamComponent<IN, OUT> {
private static final Log LOG = LogFactory.getLog(StreamTask.class);
- private AbstractRecordReader inputs;
- private List<RecordWriter<StreamRecord<OUT>>> outputs;
+ private MutableReader<IOReadableWritable> inputs;
+ MutableObjectIterator<StreamRecord<IN>> inputIter;
+ private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private StreamRecordInvokable<IN, OUT> userFunction;
private int[] numberOfOutputChannels;
private static int numTasks;
public StreamTask() {
-
- outputs = new LinkedList<RecordWriter<StreamRecord<OUT>>>();
+
+ outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
userFunction = null;
numTasks = newComponent();
instanceID = numTasks;
@@ -54,15 +59,12 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends AbstractStr
public void registerInputOutput() {
initialize();
-// try {
- setSerializers();
- setCollector();
- inputs = getConfigInputs();
- setConfigOutputs(outputs);
-// } catch (StreamComponentException e) {
-// throw new StreamComponentException("Cannot register inputs/outputs for "
-// + getClass().getSimpleName(), e);
-// }
+ setSerializers();
+ setCollector();
+ inputs = getConfigInputs();
+ setConfigOutputs(outputs);
+
+ inputIter = createInputIterator(inputs, inTupleSerializer);
numberOfOutputChannels = new int[outputs.size()];
for (int i = 0; i < numberOfOutputChannels.length; i++) {
@@ -79,6 +81,7 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends AbstractStr
Class<? extends UserTaskInvokable> userFunctionClass = configuration.getClass(
"userfunction", DefaultTaskInvokable.class, UserTaskInvokable.class);
userFunction = (UserTaskInvokable<IN, OUT>) getInvokable(userFunctionClass);
+ userFunction.initialize(collector, inputIter, inTupleSerializer);
}
@Override
@@ -87,17 +90,17 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends AbstractStr
LOG.debug("TASK " + name + " invoked with instance id " + instanceID);
}
- for (RecordWriter<StreamRecord<OUT>> output : outputs) {
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
output.initializeSerializers();
}
- invokeRecords(userFunction, inputs);
+ userFunction.invoke();
if (LOG.isDebugEnabled()) {
LOG.debug("TASK " + name + " invoke finished with instance id " + instanceID);
}
-
- for (RecordWriter<StreamRecord<OUT>> output : outputs){
+
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
output.flush();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/UnionStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/UnionStreamRecordReader.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/UnionStreamRecordReader.java
deleted file mode 100755
index 90fba85..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/UnionStreamRecordReader.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.streamcomponent;
-
-import java.io.IOException;
-
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.runtime.io.network.api.AbstractUnionRecordReader;
-import org.apache.flink.runtime.io.network.api.MutableRecordReader;
-import org.apache.flink.runtime.io.network.api.Reader;
-
-public final class UnionStreamRecordReader<T extends Tuple> extends AbstractUnionRecordReader<StreamRecord<T>>
- implements Reader<StreamRecord<T>> {
-
- @SuppressWarnings("rawtypes")
- private final Class<? extends StreamRecord> recordType;
-
- private StreamRecord<T> lookahead;
- private DeserializationDelegate<T> deserializationDelegate;
- private TupleSerializer<T> tupleSerializer;
-
- @SuppressWarnings("rawtypes")
- public UnionStreamRecordReader(MutableRecordReader<StreamRecord<T>>[] recordReaders, Class<? extends StreamRecord> class1,
- DeserializationDelegate<T> deserializationDelegate,
- TupleSerializer<T> tupleSerializer) {
- super(recordReaders);
- this.recordType = class1;
- this.deserializationDelegate = deserializationDelegate;
- this.tupleSerializer = tupleSerializer;
- }
-
- @Override
- public boolean hasNext() throws IOException, InterruptedException {
- if (this.lookahead != null) {
- return true;
- } else {
- StreamRecord<T> record = instantiateRecordType();
- record.setDeseralizationDelegate(deserializationDelegate, tupleSerializer);
- if (getNextRecord(record)) {
- this.lookahead = record;
- return true;
- } else {
- return false;
- }
- }
- }
-
- @Override
- public StreamRecord<T> next() throws IOException, InterruptedException {
- if (hasNext()) {
- StreamRecord<T> tmp = this.lookahead;
- this.lookahead = null;
- return tmp;
- } else {
- return null;
- }
- }
-
- @SuppressWarnings("unchecked")
- private StreamRecord<T> instantiateRecordType() {
- try {
- return this.recordType.newInstance();
- } catch (InstantiationException e) {
- throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName()
- + "'.", e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName()
- + "'.", e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
index a8a6012..5021013 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecord.java
@@ -19,45 +19,27 @@
package org.apache.flink.streaming.api.streamrecord;
-import java.io.IOException;
import java.io.Serializable;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
/**
* Object for wrapping a tuple with ID used for sending records between
* streaming task in Apache Flink stream processing.
*/
-public class StreamRecord<T extends Tuple> implements IOReadableWritable, Serializable {
+public class StreamRecord<T extends Tuple> implements Serializable {
private static final long serialVersionUID = 1L;
protected UID uid;
private T tuple;
- protected SerializationDelegate<T> serializationDelegate;
- protected DeserializationDelegate<T> deserializationDelegate;
protected TupleSerializer<T> tupleSerializer;
public StreamRecord() {
}
- public void setSeralizationDelegate(SerializationDelegate<T> serializationDelegate) {
- this.serializationDelegate = serializationDelegate;
- }
-
- public void setDeseralizationDelegate(DeserializationDelegate<T> deserializationDelegate,
- TupleSerializer<T> tupleSerializer) {
- this.deserializationDelegate = deserializationDelegate;
- this.tupleSerializer = tupleSerializer;
- }
-
/**
* @return The ID of the object
*/
@@ -98,22 +80,6 @@ public class StreamRecord<T extends Tuple> implements IOReadableWritable, Serial
}
@Override
- public void read(DataInputView in) throws IOException {
- uid = new UID();
- uid.read(in);
- deserializationDelegate.setInstance(tupleSerializer.createInstance());
- deserializationDelegate.read(in);
- tuple = deserializationDelegate.getInstance();
- }
-
- @Override
- public void write(DataOutputView out) throws IOException {
- uid.write(out);
- serializationDelegate.setInstance(tuple);
- serializationDelegate.write(out);
- }
-
- @Override
public String toString() {
return tuple.toString();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
new file mode 100755
index 0000000..b367333
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUStreamRecord<?>WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.streamrecord;
+
+import java.io.IOException;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+public final class StreamRecordSerializer<T extends Tuple> extends TypeSerializer<StreamRecord<T>> {
+
+ private static final long serialVersionUID = 1L;
+
+ private final TupleSerializer<T> tupleSerializer;
+
+ public StreamRecordSerializer(TupleSerializer<T> tupleSerializer) {
+ this.tupleSerializer = tupleSerializer;
+ }
+
+ @Override
+ public boolean isImmutableType() {
+ return false;
+ }
+
+ @Override
+ public boolean isStateful() {
+ return false;
+ }
+
+ @Override
+ public StreamRecord<T> createInstance() {
+ try {
+ @SuppressWarnings("unchecked")
+ StreamRecord<T> t = StreamRecord.class.newInstance().setId(0);
+ t.setTuple(tupleSerializer.createInstance());
+
+ return t;
+ } catch (Exception e) {
+ throw new RuntimeException("Cannot instantiate StreamRecord.", e);
+ }
+ }
+
+ @Override
+ public StreamRecord<T> copy(StreamRecord<T> from, StreamRecord<T> reuse) {
+
+ return null;
+ // for (int i = 0; i < arity; i++) {
+ // Object copy = fieldSerializers[i].copy(from.getField(i),
+ // reuse.getField(i));
+ // reuse.setField(copy, i);
+ // }
+ //
+ // return reuse;
+ }
+
+ @Override
+ public int getLength() {
+ return -1;
+ }
+
+ @Override
+ public void serialize(StreamRecord<T> value, DataOutputView target) throws IOException {
+ value.getId().write(target);
+ tupleSerializer.serialize(value.getTuple(), target);
+ }
+
+ @Override
+ public StreamRecord<T> deserialize(StreamRecord<T> reuse, DataInputView source)
+ throws IOException {
+ reuse.getId().read(source);
+ tupleSerializer.deserialize(reuse.getTuple(), source);
+ return reuse;
+ }
+
+ @Override
+ public void copy(DataInputView source, DataOutputView target) throws IOException {
+
+ }
+
+ // @Override
+ // public int hashCode() {
+ // int hashCode = arity * 47;
+ // for (TypeSerializer<?> ser : this.fieldSerializers) {
+ // hashCode = (hashCode << 7) | (hashCode >>> -7);
+ // hashCode += ser.hashCode();
+ // }
+ // return hashCode;
+ // }
+
+ // @Override
+ // public boolean equals(Object obj) {
+ // if (obj != null && obj instanceof StreamRecordSerializer) {
+ // StreamRecordSerializer otherTS = (StreamRecordSerializer) obj;
+ // return (otherTS.tupleClass == this.tupleClass)
+ // && Arrays.deepEquals(this.fieldSerializers, otherTS.fieldSerializers);
+ // } else {
+ // return false;
+ // }
+ // }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java
index 24893c2..06f059d 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/BroadcastPartitioner.java
@@ -20,6 +20,7 @@
package org.apache.flink.streaming.partitioner;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
/**
@@ -35,7 +36,8 @@ public class BroadcastPartitioner<T extends Tuple> implements StreamPartitioner<
boolean set;
@Override
- public int[] selectChannels(StreamRecord<T> record, int numberOfOutputChannels) {
+ public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
+ int numberOfOutputChannels) {
if (set) {
return returnArray;
} else {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
index 3ce9d19..51bb54e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/FieldsPartitioner.java
@@ -20,11 +20,12 @@
package org.apache.flink.streaming.partitioner;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
/**
- * Partitioner that selects the same (one) channel for two Tuples having a specified
- * fields equal.
+ * Partitioner that selects the same (one) channel for two Tuples having a
+ * specified fields equal.
*
* @param <T>
* Type of the Tuple
@@ -41,8 +42,9 @@ public class FieldsPartitioner<T extends Tuple> implements StreamPartitioner<T>
}
@Override
- public int[] selectChannels(StreamRecord<T> record, int numberOfOutputChannels) {
- returnArray[0] = Math.abs(record.getTuple().getField(keyPosition).hashCode())
+ public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
+ int numberOfOutputChannels) {
+ returnArray[0] = Math.abs(record.getInstance().getTuple().getField(keyPosition).hashCode())
% numberOfOutputChannels;
return returnArray;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java
index 7b26570..88da063 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ForwardPartitioner.java
@@ -20,6 +20,7 @@
package org.apache.flink.streaming.partitioner;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
/**
@@ -38,7 +39,8 @@ public class ForwardPartitioner<T extends Tuple> implements StreamPartitioner<T>
}
@Override
- public int[] selectChannels(StreamRecord<T> record, int numberOfOutputChannels) {
+ public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
+ int numberOfOutputChannels) {
return returnArray;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java
index 7cad3dd..649f4b7 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/GlobalPartitioner.java
@@ -20,6 +20,7 @@
package org.apache.flink.streaming.partitioner;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
//Group to the partitioner with the lowest id
@@ -33,7 +34,8 @@ public class GlobalPartitioner<T extends Tuple> implements StreamPartitioner<T>
}
@Override
- public int[] selectChannels(StreamRecord<T> record, int numberOfOutputChannels) {
+ public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
+ int numberOfOutputChannels) {
return returnArray;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java
index 579250e..8d292e7 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/ShufflePartitioner.java
@@ -22,12 +22,13 @@ package org.apache.flink.streaming.partitioner;
import java.util.Random;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
/**
* Partitioner that distributes the data equally by selecting one output channel
* randomly.
- *
+ *
* @param <T>
* Type of the Tuple
*/
@@ -44,7 +45,8 @@ public class ShufflePartitioner<T extends Tuple> implements StreamPartitioner<T>
}
@Override
- public int[] selectChannels(StreamRecord<T> record, int numberOfOutputChannels) {
+ public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
+ int numberOfOutputChannels) {
returnArray[0] = random.nextInt(numberOfOutputChannels);
return returnArray;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java
index 236e6b5..c0e9c1a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/StreamPartitioner.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.io.network.api.ChannelSelector;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
/**
@@ -30,6 +31,6 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T>
* Type of the Tuple
*/
-public interface StreamPartitioner<T extends Tuple> extends ChannelSelector<StreamRecord<T>>,
+public interface StreamPartitioner<T extends Tuple> extends ChannelSelector<SerializationDelegate<StreamRecord<T>>>,
Serializable {
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
index 923e4ac..c70cca3 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PrintTest.java
@@ -19,18 +19,13 @@
package org.apache.flink.streaming.api;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.flink.streaming.api.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.LogUtils;
-import org.junit.Test;
import org.apache.flink.api.java.functions.FlatMapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.util.LogUtils;
import org.apache.flink.util.Collector;
import org.apache.log4j.Level;
+import org.junit.Test;
public class PrintTest {
@@ -83,11 +78,11 @@ public class PrintTest {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
env.generateSequence(1, 10).print();
- Set<Integer> a = new HashSet<Integer>();
- a.add(-2);
- a.add(-100);
- env.fromCollection(a).print();
- env.executeTest(MEMORYSIZE);
+ env.execute();
+
}
+
+
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
index 012c89a..d8c7213 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -57,8 +57,9 @@ public class BatchReduceTest {
sum += values.next().f0;
count++;
}
-
- out.collect(new Tuple1<Double>(sum / count));
+ if(count>0){
+ out.collect(new Tuple1<Double>(sum / count));
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c1ef6d2/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
old mode 100644
new mode 100755
[29/51] [abbrv] git commit: [streaming] Added support for simple
types instead of Tuple1 in the API
Posted by se...@apache.org.
[streaming] Added support for simple types instead of Tuple1 in the API
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/1162caca
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/1162caca
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/1162caca
Branch: refs/heads/master
Commit: 1162caca857142f237c6c05a04e9f7f2afc89572
Parents: b3cd5fd
Author: gyfora <gy...@gmail.com>
Authored: Sat Aug 2 21:19:18 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:48 2014 +0200
----------------------------------------------------------------------
.../streaming/connectors/flume/FlumeSink.java | 6 +-
.../streaming/connectors/flume/FlumeSource.java | 19 +-
.../connectors/flume/FlumeTopology.java | 21 +-
.../streaming/connectors/kafka/KafkaSink.java | 3 +-
.../streaming/connectors/kafka/KafkaSource.java | 11 +-
.../streaming/connectors/rabbitmq/RMQSink.java | 13 +-
.../connectors/rabbitmq/RMQSource.java | 16 +-
.../connectors/rabbitmq/RMQTopology.java | 23 +-
.../connectors/twitter/TwitterLocal.java | 9 +-
.../connectors/twitter/TwitterSource.java | 16 +-
.../connectors/twitter/TwitterStreaming.java | 17 +-
.../apache/flink/streaming/api/DataStream.java | 20 +-
.../streaming/api/IterativeDataStream.java | 5 +-
.../flink/streaming/api/StreamConfig.java | 15 +-
.../api/StreamExecutionEnvironment.java | 48 ++--
.../flink/streaming/api/StreamOperator.java | 3 +-
.../api/collector/DirectedStreamCollector.java | 15 +-
.../streaming/api/collector/OutputSelector.java | 20 +-
.../api/collector/StreamCollector.java | 28 +--
.../api/function/co/CoMapFunction.java | 3 +-
.../api/function/sink/PrintSinkFunction.java | 3 +-
.../api/function/sink/SinkFunction.java | 5 +-
.../api/function/sink/WriteFormat.java | 4 +-
.../api/function/sink/WriteFormatAsCsv.java | 4 +-
.../api/function/sink/WriteFormatAsText.java | 4 +-
.../api/function/sink/WriteSinkFunction.java | 4 +-
.../sink/WriteSinkFunctionByBatches.java | 3 +-
.../sink/WriteSinkFunctionByMillis.java | 3 +-
.../api/function/source/FileSourceFunction.java | 9 +-
.../api/function/source/FileStreamFunction.java | 9 +-
.../function/source/FromElementsFunction.java | 9 +-
.../function/source/GenSequenceFunction.java | 7 +-
.../api/function/source/SourceFunction.java | 3 +-
.../streaming/api/invokable/SinkInvokable.java | 11 +-
.../api/invokable/SourceInvokable.java | 3 +-
.../api/invokable/StreamComponentInvokable.java | 3 +-
.../api/invokable/StreamRecordInvokable.java | 3 +-
.../api/invokable/UserTaskInvokable.java | 4 +-
.../operator/BatchReduceInvokable.java | 7 +-
.../api/invokable/operator/FilterInvokable.java | 15 +-
.../invokable/operator/FlatMapInvokable.java | 12 +-
.../api/invokable/operator/MapInvokable.java | 11 +-
.../operator/StreamReduceInvokable.java | 3 +-
.../operator/WindowReduceInvokable.java | 9 +-
.../api/invokable/operator/co/CoInvokable.java | 3 +-
.../invokable/operator/co/CoMapInvokable.java | 12 +-
.../AbstractStreamComponent.java | 19 +-
.../api/streamcomponent/CoStreamTask.java | 22 +-
.../SingleInputAbstractStreamComponent.java | 25 +-
.../streamcomponent/StreamIterationSink.java | 6 +-
.../api/streamcomponent/StreamSink.java | 7 +-
.../api/streamcomponent/StreamTask.java | 4 +-
.../api/streamrecord/StreamRecord.java | 55 ++--
.../streamrecord/StreamRecordSerializer.java | 20 +-
.../partitioner/BroadcastPartitioner.java | 3 +-
.../partitioner/DistributePartitioner.java | 3 +-
.../partitioner/FieldsPartitioner.java | 5 +-
.../partitioner/ForwardPartitioner.java | 3 +-
.../partitioner/GlobalPartitioner.java | 3 +-
.../partitioner/ShufflePartitioner.java | 3 +-
.../partitioner/StreamPartitioner.java | 3 +-
.../util/serialization/FunctionTypeWrapper.java | 17 +-
.../util/serialization/ObjectTypeWrapper.java | 24 +-
.../serialization/TypeSerializerWrapper.java | 35 ++-
.../apache/flink/streaming/api/IterateTest.java | 23 +-
.../apache/flink/streaming/api/PrintTest.java | 2 +-
.../api/collector/DirectedOutputTest.java | 33 ++-
.../api/invokable/operator/CoMapTest.java | 27 +-
.../api/invokable/operator/FilterTest.java | 23 +-
.../api/invokable/operator/FlatMapTest.java | 80 +++---
.../api/invokable/operator/MapTest.java | 250 ++++++++-----------
.../api/streamcomponent/MockRecordWriter.java | 2 +-
.../partitioner/FieldsPartitionerTest.java | 4 +-
.../serialization/TypeSerializationTest.java | 50 ++--
.../examples/wordcount/WordCountCounter.java | 7 +-
.../examples/wordcount/WordCountLocal.java | 2 +-
.../examples/wordcount/WordCountSplitter.java | 12 +-
77 files changed, 577 insertions(+), 669 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
index 20a3a4a..6f943d1 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
@@ -21,10 +21,8 @@ package org.apache.flink.streaming.connectors.flume;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.connectors.rabbitmq.RMQSource;
import org.apache.flume.Event;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.FlumeException;
@@ -32,10 +30,10 @@ import org.apache.flume.api.RpcClient;
import org.apache.flume.api.RpcClientFactory;
import org.apache.flume.event.EventBuilder;
-public abstract class FlumeSink<IN extends Tuple> extends SinkFunction<IN> {
+public abstract class FlumeSink<IN> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
- private static final Log LOG = LogFactory.getLog(RMQSource.class);
+ private static final Log LOG = LogFactory.getLog(FlumeSink.class);
private transient FlinkRpcClientFacade client;
boolean initDone = false;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
index 8b102a8..b141efb 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
@@ -21,17 +21,16 @@ package org.apache.flink.streaming.connectors.flume;
import java.util.List;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.util.Collector;
import org.apache.flume.Context;
import org.apache.flume.channel.ChannelProcessor;
import org.apache.flume.source.AvroSource;
import org.apache.flume.source.avro.AvroFlumeEvent;
import org.apache.flume.source.avro.Status;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.util.Collector;
-public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
+public abstract class FlumeSource<OUT> extends SourceFunction<OUT> {
private static final long serialVersionUID = 1L;
String host;
@@ -43,7 +42,7 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
}
public class MyAvroSource extends AvroSource {
- Collector<IN> collector;
+ Collector<OUT> collector;
/**
* Sends the AvroFlumeEvent from it's argument list to the Apache Flink
@@ -85,7 +84,7 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
*/
private void collect(AvroFlumeEvent avroEvent) {
byte[] b = avroEvent.getBody().array();
- IN tuple = FlumeSource.this.deserialize(b);
+ OUT tuple = FlumeSource.this.deserialize(b);
if (!closeWithoutSend) {
collector.collect(tuple);
}
@@ -108,7 +107,7 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
* The incoming message in a byte array
* @return The deserialized message in the required format.
*/
- public abstract IN deserialize(byte[] message);
+ public abstract OUT deserialize(byte[] message);
/**
* Configures the AvroSource. Also sets the collector so the application can
@@ -117,7 +116,7 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
* @param collector
* The collector used in the invoke function
*/
- public void configureAvroSource(Collector<IN> collector) {
+ public void configureAvroSource(Collector<OUT> collector) {
avroSource = new MyAvroSource();
avroSource.collector = collector;
@@ -138,7 +137,7 @@ public abstract class FlumeSource<IN extends Tuple> extends SourceFunction<IN> {
* The Collector for sending data to the datastream
*/
@Override
- public void invoke(Collector<IN> collector) throws Exception {
+ public void invoke(Collector<OUT> collector) throws Exception {
configureAvroSource(collector);
avroSource.start();
while (true) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
index 3c45cd4..414795b 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
@@ -20,13 +20,12 @@
package org.apache.flink.streaming.connectors.flume;
import org.apache.commons.lang.SerializationUtils;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
public class FlumeTopology {
- public static class MyFlumeSink extends FlumeSink<Tuple1<String>> {
+ public static class MyFlumeSink extends FlumeSink<String> {
private static final long serialVersionUID = 1L;
public MyFlumeSink(String host, int port) {
@@ -34,8 +33,8 @@ public class FlumeTopology {
}
@Override
- public byte[] serialize(Tuple1<String> tuple) {
- if (tuple.f0.equals("q")) {
+ public byte[] serialize(String tuple) {
+ if (tuple.equals("q")) {
try {
sendAndClose();
} catch (Exception e) {
@@ -43,12 +42,12 @@ public class FlumeTopology {
+ host, e);
}
}
- return SerializationUtils.serialize((String) tuple.getField(0));
+ return SerializationUtils.serialize(tuple);
}
}
- public static class MyFlumeSource extends FlumeSource<Tuple1<String>> {
+ public static class MyFlumeSource extends FlumeSource<String> {
private static final long serialVersionUID = 1L;
MyFlumeSource(String host, int port) {
@@ -56,14 +55,12 @@ public class FlumeTopology {
}
@Override
- public Tuple1<String> deserialize(byte[] msg) {
+ public String deserialize(byte[] msg) {
String s = (String) SerializationUtils.deserialize(msg);
- Tuple1<String> out = new Tuple1<String>();
- out.f0 = s;
if (s.equals("q")) {
closeWithoutSend();
}
- return out;
+ return s;
}
}
@@ -73,12 +70,12 @@ public class FlumeTopology {
StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
@SuppressWarnings("unused")
- DataStream<Tuple1<String>> dataStream1 = env
+ DataStream<String> dataStream1 = env
.addSource(new MyFlumeSource("localhost", 41414))
.print();
@SuppressWarnings("unused")
- DataStream<Tuple1<String>> dataStream2 = env
+ DataStream<String> dataStream2 = env
.fromElements("one", "two", "three", "four", "five", "q")
.addSink(new MyFlumeSink("localhost", 42424));
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
index 7e3f3db..955e8dc 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSink.java
@@ -25,10 +25,9 @@ import kafka.javaapi.producer.Producer;
import kafka.producer.KeyedMessage;
import kafka.producer.ProducerConfig;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
-public abstract class KafkaSink<IN extends Tuple, OUT> extends SinkFunction<IN> {
+public abstract class KafkaSink<IN, OUT> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
private kafka.javaapi.producer.Producer<Integer, OUT> producer;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
index 623e3b8..228069a 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSource.java
@@ -29,12 +29,11 @@ import kafka.consumer.ConsumerIterator;
import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
-public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
+public abstract class KafkaSource<OUT> extends SourceFunction<OUT> {
private static final long serialVersionUID = 1L;
private final String zkQuorum;
@@ -45,7 +44,7 @@ public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
private boolean closeWithoutSend = false;
private boolean sendAndClose = false;
- IN outTuple;
+ OUT outTuple;
public KafkaSource(String zkQuorum, String groupId, String topicId, int numThreads) {
this.zkQuorum = zkQuorum;
@@ -74,7 +73,7 @@ public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
* The Collector for sending data to the dataStream
*/
@Override
- public void invoke(Collector<IN> collector) throws Exception {
+ public void invoke(Collector<OUT> collector) throws Exception {
initializeConnection();
Map<String, Integer> topicCountMap = new HashMap<String, Integer>();
@@ -85,7 +84,7 @@ public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
ConsumerIterator<byte[], byte[]> it = stream.iterator();
while (it.hasNext()) {
- IN out = deserialize(it.next().message());
+ OUT out = deserialize(it.next().message());
if (closeWithoutSend) {
break;
}
@@ -104,7 +103,7 @@ public abstract class KafkaSource<IN extends Tuple> extends SourceFunction<IN> {
* The incoming message in a byte array
* @return The deserialized message in the required format.
*/
- public abstract IN deserialize(byte[] message);
+ public abstract OUT deserialize(byte[] message);
/**
* Closes the connection immediately and no further data will be sent.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index d7ed17a..4225cd3 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -21,16 +21,15 @@ package org.apache.flink.streaming.connectors.rabbitmq;
import java.io.IOException;
-import com.rabbitmq.client.Channel;
-import com.rabbitmq.client.Connection;
-import com.rabbitmq.client.ConnectionFactory;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
-public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+
+public abstract class RMQSink<IN> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
private static final Log LOG = LogFactory.getLog(RMQSource.class);
@@ -103,7 +102,7 @@ public abstract class RMQSink<IN extends Tuple> extends SinkFunction<IN> {
* The tuple used for the serialization
* @return The serialized byte array.
*/
- public abstract byte[] serialize(Tuple tuple);
+ public abstract byte[] serialize(IN tuple);
/**
* Closes the connection.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index dfea55a..8303b1a 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -23,18 +23,16 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.util.Collector;
import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
import com.rabbitmq.client.ConnectionFactory;
import com.rabbitmq.client.QueueingConsumer;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.DataStream;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.util.Collector;
-
-public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
+public abstract class RMQSource<OUT> extends SourceFunction<OUT> {
private static final long serialVersionUID = 1L;
private static final Log LOG = LogFactory.getLog(RMQSource.class);
@@ -50,7 +48,7 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
private transient QueueingConsumer consumer;
private transient QueueingConsumer.Delivery delivery;
- IN outTuple;
+ OUT outTuple;
public RMQSource(String HOST_NAME, String QUEUE_NAME) {
this.HOST_NAME = HOST_NAME;
@@ -82,7 +80,7 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
* The Collector for sending data to the dataStream
*/
@Override
- public void invoke(Collector<IN> collector) throws Exception {
+ public void invoke(Collector<OUT> collector) throws Exception {
initializeConnection();
while (true) {
@@ -122,7 +120,7 @@ public abstract class RMQSource<IN extends Tuple> extends SourceFunction<IN> {
* The incoming message in a byte array
* @return The deserialized message in the required format.
*/
- public abstract IN deserialize(byte[] message);
+ public abstract OUT deserialize(byte[] message);
/**
* Closes the connection immediately and no further data will be sent.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
index 828c2fa..94ae43f 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
@@ -20,15 +20,12 @@
package org.apache.flink.streaming.connectors.rabbitmq;
import org.apache.commons.lang.SerializationUtils;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
public class RMQTopology {
- public static final class MyRMQSink extends RMQSink<Tuple1<String>> {
+ public static final class MyRMQSink extends RMQSink<String> {
public MyRMQSink(String HOST_NAME, String QUEUE_NAME) {
super(HOST_NAME, QUEUE_NAME);
}
@@ -36,16 +33,16 @@ public class RMQTopology {
private static final long serialVersionUID = 1L;
@Override
- public byte[] serialize(Tuple t) {
- if (t.getField(0).equals("q")) {
+ public byte[] serialize(String t) {
+ if (t.equals("q")) {
sendAndClose();
}
- return SerializationUtils.serialize((String) t.getField(0));
+ return SerializationUtils.serialize((String) t);
}
}
- public static final class MyRMQSource extends RMQSource<Tuple1<String>> {
+ public static final class MyRMQSource extends RMQSource<String> {
public MyRMQSource(String HOST_NAME, String QUEUE_NAME) {
super(HOST_NAME, QUEUE_NAME);
@@ -54,14 +51,12 @@ public class RMQTopology {
private static final long serialVersionUID = 1L;
@Override
- public Tuple1<String> deserialize(byte[] t) {
+ public String deserialize(byte[] t) {
String s = (String) SerializationUtils.deserialize(t);
- Tuple1<String> out = new Tuple1<String>();
- out.f0 = s;
if (s.equals("q")) {
closeWithoutSend();
}
- return out;
+ return s;
}
}
@@ -71,12 +66,12 @@ public class RMQTopology {
StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
@SuppressWarnings("unused")
- DataStream<Tuple1<String>> dataStream1 = env
+ DataStream<String> dataStream1 = env
.addSource(new MyRMQSource("localhost", "hello"))
.print();
@SuppressWarnings("unused")
- DataStream<Tuple1<String>> dataStream2 = env
+ DataStream<String> dataStream2 = env
.fromElements("one", "two", "three", "four", "five", "q")
.addSink(new MyRMQSink("localhost", "hello"));
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
index 138fe05..cb868f5 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.connectors.twitter;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
@@ -41,7 +40,7 @@ public class TwitterLocal {
* FlatMapFunction to determine the language of tweets if possible
*/
public static class SelectLanguageFlatMap extends
- JSONParseFlatMap<Tuple1<String>, Tuple1<String>> {
+ JSONParseFlatMap<String, String> {
private static final long serialVersionUID = 1L;
@@ -49,9 +48,9 @@ public class TwitterLocal {
* Select the language from the incoming JSON text
*/
@Override
- public void flatMap(Tuple1<String> value, Collector<Tuple1<String>> out) throws Exception {
+ public void flatMap(String value, Collector<String> out) throws Exception {
- out.collect(new Tuple1<String>(colationOfNull(getField(value.f0, "lang"))));
+ out.collect(colationOfNull(getField(value, "lang")));
}
/**
@@ -81,7 +80,7 @@ public class TwitterLocal {
StreamExecutionEnvironment env = StreamExecutionEnvironment
.createLocalEnvironment(PARALLELISM);
- DataStream<Tuple1<String>> streamSource = env.addSource(new TwitterSource(path, 100),
+ DataStream<String> streamSource = env.addSource(new TwitterSource(path, 100),
SOURCE_PARALLELISM);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
index bbff732..bc0995d 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
@@ -29,8 +29,6 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
@@ -46,9 +44,9 @@ import com.twitter.hbc.httpclient.auth.OAuth1;
* Implementation of {@link SourceFunction} specialized to emit tweets from Twitter.
* It can connect to Twitter Streaming API, collect tweets and
*/
-public class TwitterSource extends SourceFunction<Tuple1<String>> {
+public class TwitterSource extends SourceFunction<String> {
- private static final Log LOG = LogFactory.getLog(DataStream.class);
+ private static final Log LOG = LogFactory.getLog(TwitterSource.class);
private static final long serialVersionUID = 1L;
private String authPath;
@@ -88,7 +86,7 @@ public class TwitterSource extends SourceFunction<Tuple1<String>> {
*
*/
@Override
- public void invoke(Collector<Tuple1<String>> collector) throws Exception {
+ public void invoke(Collector<String> collector) throws Exception {
initializeConnection();
@@ -169,7 +167,7 @@ public class TwitterSource extends SourceFunction<Tuple1<String>> {
* @param collector
* @param piece
*/
- protected void collectMessages(Collector<Tuple1<String>> collector, int piece) {
+ protected void collectMessages(Collector<String> collector, int piece) {
if (LOG.isInfoEnabled()) {
LOG.info("Collecting tweets");
@@ -189,7 +187,7 @@ public class TwitterSource extends SourceFunction<Tuple1<String>> {
* @param collector
*
*/
- protected void collectMessages(Collector<Tuple1<String>> collector) {
+ protected void collectMessages(Collector<String> collector) {
if (LOG.isInfoEnabled()) {
LOG.info("Tweet-stream begins");
@@ -204,7 +202,7 @@ public class TwitterSource extends SourceFunction<Tuple1<String>> {
* Put one tweet into the collector.
* @param collector
*/
- protected void collectOneMessage(Collector<Tuple1<String>> collector) {
+ protected void collectOneMessage(Collector<String> collector) {
if (client.isDone()) {
if (LOG.isErrorEnabled()) {
LOG.error("Client connection closed unexpectedly: "
@@ -215,7 +213,7 @@ public class TwitterSource extends SourceFunction<Tuple1<String>> {
try {
String msg = queue.poll(waitSec, TimeUnit.SECONDS);
if (msg != null) {
- collector.collect(new Tuple1<String>(msg));
+ collector.collect(msg);
} else {
if (LOG.isInfoEnabled()) {
LOG.info("Did not receive a message in " + waitSec
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
index 805bf06..ee986ea 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.connectors.twitter;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
@@ -47,21 +46,21 @@ public class TwitterStreaming {
}
public static class SelectDataFlatMap extends
- JSONParseFlatMap<Tuple1<String>, Tuple5<Long, Long, String, String, String>> {
+ JSONParseFlatMap<String, Tuple5<Long, Long, String, String, String>> {
private static final long serialVersionUID = 1L;
@Override
- public void flatMap(Tuple1<String> value,
+ public void flatMap(String value,
Collector<Tuple5<Long, Long, String, String, String>> out)
throws Exception {
out.collect(new Tuple5<Long, Long, String, String, String>(
- convertDateString2Long(getField(value.f0, "id")),
- convertDateString2LongDate(getField(value.f0, "created_at")),
- colationOfNull(getField(value.f0, "user.name")),
- colationOfNull(getField(value.f0, "text")),
- getField(value.f0, "lang")));
+ convertDateString2Long(getField(value, "id")),
+ convertDateString2LongDate(getField(value, "created_at")),
+ colationOfNull(getField(value, "user.name")),
+ colationOfNull(getField(value, "text")),
+ getField(value, "lang")));
}
protected String colationOfNull(String in){
@@ -94,7 +93,7 @@ public class TwitterStreaming {
StreamExecutionEnvironment env = StreamExecutionEnvironment
.createLocalEnvironment(PARALLELISM);
- DataStream<Tuple1<String>> streamSource = env.addSource(
+ DataStream<String> streamSource = env.addSource(
new TwitterSource(path,100), SOURCE_PARALLELISM);
DataStream<Tuple5<Long, Long, String, String, String>> selectedDataStream = streamSource
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index 1bde6a6..d0f1294 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -70,7 +70,7 @@ import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
* The type of the DataStream, i.e., the type of the elements of the
* DataStream.
*/
-public class DataStream<T extends Tuple> {
+public class DataStream<T> {
protected static Integer counter = 0;
protected final StreamExecutionEnvironment environment;
@@ -352,7 +352,7 @@ public class DataStream<T extends Tuple> {
* output type
* @return The transformed DataStream.
*/
- public <R extends Tuple> StreamOperator<T, R> map(MapFunction<T, R> mapper) {
+ public <R> StreamOperator<T, R> map(MapFunction<T, R> mapper) {
return addFunction("map", mapper, new FunctionTypeWrapper<T, Tuple, R>(mapper,
MapFunction.class, 0, -1, 1), new MapInvokable<T, R>(mapper));
}
@@ -372,7 +372,7 @@ public class DataStream<T extends Tuple> {
* {@link CoMapFunction#map2(Tuple)}
* @return The transformed DataStream
*/
- public <T2 extends Tuple, R extends Tuple> DataStream<R> coMapWith(
+ public <T2, R> DataStream<R> coMapWith(
CoMapFunction<T, T2, R> coMapper, DataStream<T2> otherStream) {
return addCoFunction("coMap", new DataStream<T>(this), new DataStream<T2>(otherStream),
coMapper,
@@ -394,7 +394,7 @@ public class DataStream<T extends Tuple> {
* output type
* @return The transformed DataStream.
*/
- public <R extends Tuple> StreamOperator<T, R> flatMap(FlatMapFunction<T, R> flatMapper) {
+ public <R> StreamOperator<T, R> flatMap(FlatMapFunction<T, R> flatMapper) {
return addFunction("flatMap", flatMapper, new FunctionTypeWrapper<T, Tuple, R>(flatMapper,
FlatMapFunction.class, 0, -1, 1), new FlatMapInvokable<T, R>(flatMapper));
}
@@ -430,7 +430,7 @@ public class DataStream<T extends Tuple> {
* output type
* @return The modified DataStream.
*/
- public <R extends Tuple> StreamOperator<T, R> batchReduce(GroupReduceFunction<T, R> reducer,
+ public <R> StreamOperator<T, R> batchReduce(GroupReduceFunction<T, R> reducer,
int batchSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<T, R>(reducer,
@@ -453,7 +453,7 @@ public class DataStream<T extends Tuple> {
* output type
* @return The modified DataStream.
*/
- public <R extends Tuple> StreamOperator<T, R> windowReduce(GroupReduceFunction<T, R> reducer,
+ public <R> StreamOperator<T, R> windowReduce(GroupReduceFunction<T, R> reducer,
long windowSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<T, R>(reducer,
@@ -476,7 +476,7 @@ public class DataStream<T extends Tuple> {
* type of the return stream
* @return the data stream constructed
*/
- private <R extends Tuple> StreamOperator<T, R> addFunction(String functionName,
+ private <R> StreamOperator<T, R> addFunction(String functionName,
final AbstractFunction function, TypeSerializerWrapper<T, Tuple, R> typeWrapper,
UserTaskInvokable<T, R> functionInvokable) {
@@ -500,7 +500,7 @@ public class DataStream<T extends Tuple> {
return returnStream;
}
- protected <T1 extends Tuple, T2 extends Tuple, R extends Tuple> DataStream<R> addCoFunction(
+ protected <T1, T2, R> DataStream<R> addCoFunction(
String functionName, DataStream<T1> inputStream1, DataStream<T2> inputStream2,
final AbstractFunction function, TypeSerializerWrapper<T1, T2, R> typeWrapper,
CoInvokable<T1, T2, R> functionInvokable) {
@@ -535,7 +535,7 @@ public class DataStream<T extends Tuple> {
* @param typeNumber
* Number of the type (used at co-functions)
*/
- <X extends Tuple> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
+ <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
for (int i = 0; i < inputStream.connectIDs.size(); i++) {
String inputID = inputStream.connectIDs.get(i);
StreamPartitioner<X> partitioner = inputStream.partitioners.get(i);
@@ -926,7 +926,7 @@ public class DataStream<T extends Tuple> {
return new IterativeDataStream<T>(this);
}
- protected <R extends Tuple> DataStream<T> addIterationSource(String iterationID) {
+ protected <R> DataStream<T> addIterationSource(String iterationID) {
DataStream<R> returnStream = new DataStream<R>(environment, "iterationSource");
jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
index 1cfb625..bfce834 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.api;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.partitioner.ForwardPartitioner;
/**
@@ -29,7 +28,7 @@ import org.apache.flink.streaming.partitioner.ForwardPartitioner;
* @param <T>
* Type of the DataStream
*/
-public class IterativeDataStream<T extends Tuple> extends StreamOperator<T, T> {
+public class IterativeDataStream<T> extends StreamOperator<T, T> {
static Integer iterationCount = 0;
@@ -69,7 +68,7 @@ public class IterativeDataStream<T extends Tuple> extends StreamOperator<T, T> {
* when used with directed emits
*
*/
- public <R extends Tuple> DataStream<T> closeWith(DataStream<T> iterationResult,
+ public <R> DataStream<T> closeWith(DataStream<T> iterationResult,
String iterationName) {
DataStream<R> returnStream = new DataStream<R>(environment, "iterationSink");
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index 3d49928..a102a00 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -24,7 +24,6 @@ import java.io.IOException;
import org.apache.commons.lang3.SerializationException;
import org.apache.commons.lang3.SerializationUtils;
import org.apache.flink.api.common.functions.AbstractFunction;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
@@ -74,12 +73,12 @@ public class StreamConfig {
// CONFIGS
public void setTypeWrapper(
- TypeSerializerWrapper<? extends Tuple, ? extends Tuple, ? extends Tuple> typeWrapper) {
+ TypeSerializerWrapper<?, ?, ?> typeWrapper) {
config.setBytes("typeWrapper", SerializationUtils.serialize(typeWrapper));
}
@SuppressWarnings("unchecked")
- public <IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> TypeSerializerWrapper<IN1, IN2, OUT> getTypeWrapper() {
+ public <IN1, IN2, OUT> TypeSerializerWrapper<IN1, IN2, OUT> getTypeWrapper() {
byte[] serializedWrapper = config.getBytes("typeWrapper", null);
if (serializedWrapper == null) {
@@ -106,7 +105,7 @@ public class StreamConfig {
return config.getLong(BUFFER_TIMEOUT, DEFAULT_TIMEOUT);
}
- public void setUserInvokable(StreamComponentInvokable<? extends Tuple> invokableObject) {
+ public void setUserInvokable(StreamComponentInvokable<?> invokableObject) {
if (invokableObject != null) {
config.setClass(USER_FUNCTION, invokableObject.getClass());
@@ -125,7 +124,7 @@ public class StreamConfig {
// return (Class<? extends T>) config.getClass(USER_FUNCTION, null);
// }
- public <T extends Tuple> StreamComponentInvokable<T> getUserInvokableObject() {
+ public <T> StreamComponentInvokable<T> getUserInvokableObject() {
try {
return deserializeObject(config.getBytes(SERIALIZEDUDF, null));
} catch (Exception e) {
@@ -186,7 +185,7 @@ public class StreamConfig {
}
}
- public <T extends Tuple> OutputSelector<T> getOutputSelector() {
+ public <T> OutputSelector<T> getOutputSelector() {
try {
return deserializeObject(config.getBytes(OUTPUT_SELECTOR, null));
} catch (Exception e) {
@@ -211,14 +210,14 @@ public class StreamConfig {
return config.getInteger(NUMBER_OF_OUTPUT_CHANNELS + outputIndex, 0);
}
- public <T extends Tuple> void setPartitioner(int outputIndex,
+ public <T> void setPartitioner(int outputIndex,
StreamPartitioner<T> partitionerObject) {
config.setBytes(PARTITIONER_OBJECT + outputIndex,
SerializationUtils.serialize(partitionerObject));
}
- public <T extends Tuple> StreamPartitioner<T> getPartitioner(int outputIndex)
+ public <T> StreamPartitioner<T> getPartitioner(int outputIndex)
throws ClassNotFoundException, IOException {
return deserializeObject(config.getBytes(PARTITIONER_OBJECT + outputIndex,
SerializationUtils.serialize(new ShufflePartitioner<T>())));
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index 4539126..0e77912 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -26,7 +26,6 @@ import org.apache.commons.lang3.SerializationException;
import org.apache.commons.lang3.SerializationUtils;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.function.source.FileSourceFunction;
import org.apache.flink.streaming.api.function.source.FileStreamFunction;
import org.apache.flink.streaming.api.function.source.FromElementsFunction;
@@ -152,11 +151,11 @@ public abstract class StreamExecutionEnvironment {
* "file:///some/local/file" or "hdfs://host:port/file/path").
* @return The DataStream representing the text file.
*/
- public DataStream<Tuple1<String>> readTextFile(String filePath) {
+ public DataStream<String> readTextFile(String filePath) {
return addSource(new FileSourceFunction(filePath), 1);
}
- public DataStream<Tuple1<String>> readTextFile(String filePath, int parallelism) {
+ public DataStream<String> readTextFile(String filePath, int parallelism) {
return addSource(new FileSourceFunction(filePath), parallelism);
}
@@ -170,11 +169,11 @@ public abstract class StreamExecutionEnvironment {
* "file:///some/local/file" or "hdfs://host:port/file/path").
* @return The DataStream representing the text file.
*/
- public DataStream<Tuple1<String>> readTextStream(String filePath) {
+ public DataStream<String> readTextStream(String filePath) {
return addSource(new FileStreamFunction(filePath), 1);
}
- public DataStream<Tuple1<String>> readTextStream(String filePath, int parallelism) {
+ public DataStream<String> readTextStream(String filePath, int parallelism) {
return addSource(new FileStreamFunction(filePath), parallelism);
}
@@ -191,15 +190,14 @@ public abstract class StreamExecutionEnvironment {
* type of the returned stream
* @return The DataStream representing the elements.
*/
- public <X extends Serializable> DataStream<Tuple1<X>> fromElements(X... data) {
- DataStream<Tuple1<X>> returnStream = new DataStream<Tuple1<X>>(this, "elements");
+ public <X extends Serializable> DataStream<X> fromElements(X... data) {
+ DataStream<X> returnStream = new DataStream<X>(this, "elements");
try {
- SourceFunction<Tuple1<X>> function = new FromElementsFunction<X>(data);
- jobGraphBuilder.addSource(returnStream.getId(),
- new SourceInvokable<Tuple1<X>>(function),
- new ObjectTypeWrapper<Tuple1<X>, Tuple, Tuple1<X>>(data[0], null, data[0]),
- "source", SerializationUtils.serialize(function), 1);
+ SourceFunction<X> function = new FromElementsFunction<X>(data);
+ jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<X>(function),
+ new ObjectTypeWrapper<X, Tuple, X>(data[0], null, data[0]), "source",
+ SerializationUtils.serialize(function), 1);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize elements");
}
@@ -218,22 +216,22 @@ public abstract class StreamExecutionEnvironment {
* type of the returned stream
* @return The DataStream representing the elements.
*/
- public <X extends Serializable> DataStream<Tuple1<X>> fromCollection(Collection<X> data) {
- DataStream<Tuple1<X>> returnStream = new DataStream<Tuple1<X>>(this, "elements");
+ @SuppressWarnings("unchecked")
+ public <X extends Serializable> DataStream<X> fromCollection(Collection<X> data) {
+ DataStream<X> returnStream = new DataStream<X>(this, "elements");
if (data.isEmpty()) {
throw new RuntimeException("Collection must not be empty");
}
try {
- SourceFunction<Tuple1<X>> function = new FromElementsFunction<X>(data);
-
- jobGraphBuilder
- .addSource(returnStream.getId(), new SourceInvokable<Tuple1<X>>(
- new FromElementsFunction<X>(data)),
- new ObjectTypeWrapper<Tuple1<X>, Tuple, Tuple1<X>>(data.toArray()[0],
- null, data.toArray()[0]), "source", SerializationUtils
- .serialize(function), 1);
+ SourceFunction<X> function = new FromElementsFunction<X>(data);
+
+ jobGraphBuilder.addSource(
+ returnStream.getId(),
+ new SourceInvokable<X>(new FromElementsFunction<X>(data)),
+ new ObjectTypeWrapper<X, Tuple, X>((X) data.toArray()[0], null, (X) data
+ .toArray()[0]), "source", SerializationUtils.serialize(function), 1);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize collection");
}
@@ -250,7 +248,7 @@ public abstract class StreamExecutionEnvironment {
* The number to stop at (inclusive)
* @return A DataStrean, containing all number in the [from, to] interval.
*/
- public DataStream<Tuple1<Long>> generateSequence(long from, long to) {
+ public DataStream<Long> generateSequence(long from, long to) {
return addSource(new GenSequenceFunction(from, to), 1);
}
@@ -265,7 +263,7 @@ public abstract class StreamExecutionEnvironment {
* type of the returned stream
* @return the data stream constructed
*/
- public <T extends Tuple> DataStream<T> addSource(SourceFunction<T> function, int parallelism) {
+ public <T> DataStream<T> addSource(SourceFunction<T> function, int parallelism) {
DataStream<T> returnStream = new DataStream<T>(this, "source");
try {
@@ -279,7 +277,7 @@ public abstract class StreamExecutionEnvironment {
return returnStream;
}
- public <T extends Tuple> DataStream<T> addSource(SourceFunction<T> sourceFunction) {
+ public <T> DataStream<T> addSource(SourceFunction<T> sourceFunction) {
return addSource(sourceFunction, 1);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
index d8adb97..7edde1c 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
@@ -19,9 +19,8 @@
package org.apache.flink.streaming.api;
-import org.apache.flink.api.java.tuple.Tuple;
-public class StreamOperator<IN extends Tuple, OUT extends Tuple> extends DataStream<OUT> {
+public class StreamOperator<IN, OUT > extends DataStream<OUT> {
protected StreamOperator(StreamExecutionEnvironment environment, String operatorType) {
super(environment, operatorType);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
index f968b83..73a5749 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
@@ -23,7 +23,6 @@ import java.util.Collection;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -36,7 +35,7 @@ import org.apache.flink.util.StringUtils;
* @param <T>
* Type of the Tuple collected.
*/
-public class DirectedStreamCollector<T extends Tuple> extends StreamCollector<T> {
+public class DirectedStreamCollector<T> extends StreamCollector<T> {
OutputSelector<T> outputSelector;
private static final Log log = LogFactory.getLog(DirectedStreamCollector.class);
@@ -47,7 +46,7 @@ public class DirectedStreamCollector<T extends Tuple> extends StreamCollector<T>
* @param channelID
* Channel ID of the Task
* @param serializationDelegate
- * Serialization delegate used for tuple serialization
+ * Serialization delegate used for serialization
* @param outputSelector
* User defined {@link OutputSelector}
*/
@@ -63,12 +62,12 @@ public class DirectedStreamCollector<T extends Tuple> extends StreamCollector<T>
* Collects and emits a tuple to the outputs by reusing a StreamRecord
* object.
*
- * @param tuple
- * Tuple to be collected and emitted.
+ * @param outputObject
+ * Object to be collected and emitted.
*/
@Override
- public void collect(T tuple) {
- streamRecord.setTuple(tuple);
+ public void collect(T outputObject) {
+ streamRecord.setObject(outputObject);
emit(streamRecord);
}
@@ -80,7 +79,7 @@ public class DirectedStreamCollector<T extends Tuple> extends StreamCollector<T>
* Record to emit.
*/
private void emit(StreamRecord<T> streamRecord) {
- Collection<String> outputNames = outputSelector.getOutputs(streamRecord.getTuple());
+ Collection<String> outputNames = outputSelector.getOutputs(streamRecord.getObject());
streamRecord.setId(channelID);
serializationDelegate.setInstance(streamRecord);
for (String outputName : outputNames) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
index c4262b6..6d63385 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
@@ -23,17 +23,15 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
-import org.apache.flink.api.java.tuple.Tuple;
-
/**
* Class for defining an OutputSelector for the directTo operator. Every output
- * tuple of a directed DataStream will run through this operator to select
+ * object of a directed DataStream will run through this operator to select
* outputs.
*
* @param <T>
- * Type parameter of the directed tuples.
+ * Type parameter of the directed tuples/objects.
*/
-public abstract class OutputSelector<T extends Tuple> implements Serializable {
+public abstract class OutputSelector<T> implements Serializable {
private static final long serialVersionUID = 1L;
private Collection<String> outputs;
@@ -42,21 +40,21 @@ public abstract class OutputSelector<T extends Tuple> implements Serializable {
outputs = new ArrayList<String>();
}
- Collection<String> getOutputs(T tuple) {
+ Collection<String> getOutputs(T outputObject) {
outputs.clear();
- select(tuple, outputs);
+ select(outputObject, outputs);
return outputs;
}
/**
- * Method for selecting output names for the emitted tuples when using the
+ * Method for selecting output names for the emitted objects when using the
* directTo operator. The tuple will be emitted only to output names which
* are added to the outputs collection.
*
- * @param tuple
- * Tuple for which the output selection should be made.
+ * @param outputObject
+ * Output object for which the output selection should be made.
* @param outputs
* Selected output names should be added to this collection.
*/
- public abstract void select(T tuple, Collection<String> outputs);
+ public abstract void select(T outputObject, Collection<String> outputs);
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
index 544a695..4317f75 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
@@ -26,7 +26,6 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -34,14 +33,14 @@ import org.apache.flink.util.Collector;
import org.apache.flink.util.StringUtils;
/**
- * Collector for tuples in Apache Flink stream processing. The collected tuples
- * will be wrapped with ID in a {@link StreamRecord} and then emitted to the
- * outputs.
+ * Collector for tuples in Apache Flink stream processing. The collected
+ * tuples/obecjts will be wrapped with ID in a {@link StreamRecord} and then
+ * emitted to the outputs.
*
* @param <T>
- * Type of the Tuple collected.
+ * Type of the Tuples/Objects collected.
*/
-public class StreamCollector<T extends Tuple> implements Collector<T> {
+public class StreamCollector<T> implements Collector<T> {
private static final Log LOG = LogFactory.getLog(StreamCollector.class);
@@ -57,9 +56,10 @@ public class StreamCollector<T extends Tuple> implements Collector<T> {
* @param channelID
* Channel ID of the Task
* @param serializationDelegate
- * Serialization delegate used for tuple serialization
+ * Serialization delegate used for serialization
*/
- public StreamCollector(int channelID, SerializationDelegate<StreamRecord<T>> serializationDelegate) {
+ public StreamCollector(int channelID,
+ SerializationDelegate<StreamRecord<T>> serializationDelegate) {
this.serializationDelegate = serializationDelegate;
this.streamRecord = new StreamRecord<T>();
@@ -92,15 +92,15 @@ public class StreamCollector<T extends Tuple> implements Collector<T> {
}
/**
- * Collects and emits a tuple to the outputs by reusing a StreamRecord
- * object.
+ * Collects and emits a tuple/object to the outputs by reusing a
+ * StreamRecord object.
*
- * @param tuple
- * Tuple to be collected and emitted.
+ * @param outputObject
+ * Object to be collected and emitted.
*/
@Override
- public void collect(T tuple) {
- streamRecord.setTuple(tuple);
+ public void collect(T outputObject) {
+ streamRecord.setObject(outputObject);
emit(streamRecord);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
index 5885cbf..6e4d877 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
@@ -19,9 +19,8 @@
package org.apache.flink.streaming.api.function.co;
import org.apache.flink.api.common.functions.AbstractFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-public abstract class CoMapFunction<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends AbstractFunction {
+public abstract class CoMapFunction<IN1, IN2, OUT> extends AbstractFunction {
private static final long serialVersionUID = 1L;
public abstract OUT map1(IN1 value);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
index 7918e48..026c18e 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.api.function.sink;
-import org.apache.flink.api.java.tuple.Tuple;
/**
* Dummy implementation of the SinkFunction writing every tuple to the standard
@@ -28,7 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple;
* @param <IN>
* Input tuple type
*/
-public class PrintSinkFunction<IN extends Tuple> extends SinkFunction<IN> {
+public class PrintSinkFunction<IN> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
index cc4fb96..867c9f8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/SinkFunction.java
@@ -22,12 +22,11 @@ package org.apache.flink.streaming.api.function.sink;
import java.io.Serializable;
import org.apache.flink.api.common.functions.AbstractFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-public abstract class SinkFunction<IN extends Tuple> extends AbstractFunction implements Serializable {
+public abstract class SinkFunction<IN> extends AbstractFunction implements Serializable {
private static final long serialVersionUID = 1L;
- public abstract void invoke(IN tuple);
+ public abstract void invoke(IN value);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormat.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormat.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormat.java
index 18853b3..3e93a97 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormat.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormat.java
@@ -22,8 +22,6 @@ package org.apache.flink.streaming.api.function.sink;
import java.io.Serializable;
import java.util.ArrayList;
-import org.apache.flink.api.java.tuple.Tuple;
-
/**
* Abstract class for formatting the output of the writeAsText and writeAsCsv
* functions.
@@ -31,7 +29,7 @@ import org.apache.flink.api.java.tuple.Tuple;
* @param <IN>
* Input tuple type
*/
-public abstract class WriteFormat<IN extends Tuple> implements Serializable {
+public abstract class WriteFormat<IN> implements Serializable {
private static final long serialVersionUID = 1L;
/**
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsCsv.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsCsv.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsCsv.java
index e10a9c8..5fa099f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsCsv.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsCsv.java
@@ -25,15 +25,13 @@ import java.io.IOException;
import java.io.PrintWriter;
import java.util.ArrayList;
-import org.apache.flink.api.java.tuple.Tuple;
-
/**
* Writes tuples in csv format.
*
* @param <IN>
* Input tuple type
*/
-public class WriteFormatAsCsv<IN extends Tuple> extends WriteFormat<IN> {
+public class WriteFormatAsCsv<IN> extends WriteFormat<IN> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsText.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsText.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsText.java
index 2d591ae..6a82877 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsText.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteFormatAsText.java
@@ -25,15 +25,13 @@ import java.io.IOException;
import java.io.PrintWriter;
import java.util.ArrayList;
-import org.apache.flink.api.java.tuple.Tuple;
-
/**
* Writes tuples in text format.
*
* @param <IN>
* Input tuple type
*/
-public class WriteFormatAsText<IN extends Tuple> extends WriteFormat<IN> {
+public class WriteFormatAsText<IN> extends WriteFormat<IN> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
index d473190..774dd63 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunction.java
@@ -23,8 +23,6 @@ import java.io.FileNotFoundException;
import java.io.PrintWriter;
import java.util.ArrayList;
-import org.apache.flink.api.java.tuple.Tuple;
-
/**
* Simple implementation of the SinkFunction writing tuples as simple text to
* the file specified by path. Tuples are collected to a list and written to the
@@ -34,7 +32,7 @@ import org.apache.flink.api.java.tuple.Tuple;
* @param <IN>
* Input tuple type
*/
-public abstract class WriteSinkFunction<IN extends Tuple> extends SinkFunction<IN> {
+public abstract class WriteSinkFunction<IN> extends SinkFunction<IN> {
private static final long serialVersionUID = 1L;
protected final String path;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByBatches.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByBatches.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByBatches.java
index 3797d13..c860c52 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByBatches.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByBatches.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.api.function.sink;
-import org.apache.flink.api.java.tuple.Tuple;
/**
* Implementation of WriteSinkFunction. Writes tuples to file in equally sized
@@ -28,7 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple;
* @param <IN>
* Input tuple type
*/
-public class WriteSinkFunctionByBatches<IN extends Tuple> extends WriteSinkFunction<IN> {
+public class WriteSinkFunctionByBatches<IN> extends WriteSinkFunction<IN> {
private static final long serialVersionUID = 1L;
private final int batchSize;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByMillis.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByMillis.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByMillis.java
index cb77e6d..9271f36 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByMillis.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/WriteSinkFunctionByMillis.java
@@ -19,7 +19,6 @@
package org.apache.flink.streaming.api.function.sink;
-import org.apache.flink.api.java.tuple.Tuple;
/**
* Implementation of WriteSinkFunction. Writes tuples to file in every millis
@@ -28,7 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple;
* @param <IN>
* Input tuple type
*/
-public class WriteSinkFunctionByMillis<IN extends Tuple> extends WriteSinkFunction<IN> {
+public class WriteSinkFunctionByMillis<IN> extends WriteSinkFunction<IN> {
private static final long serialVersionUID = 1L;
private final long millis;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
index f6c2c72..3a732be 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
@@ -23,27 +23,24 @@ import java.io.BufferedReader;
import java.io.FileReader;
import java.io.IOException;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
-public class FileSourceFunction extends SourceFunction<Tuple1<String>> {
+public class FileSourceFunction extends SourceFunction<String> {
private static final long serialVersionUID = 1L;
private final String path;
- private Tuple1<String> outTuple = new Tuple1<String>();
public FileSourceFunction(String path) {
this.path = path;
}
@Override
- public void invoke(Collector<Tuple1<String>> collector) throws IOException {
+ public void invoke(Collector<String> collector) throws IOException {
BufferedReader br = new BufferedReader(new FileReader(path));
String line = br.readLine();
while (line != null) {
if (line != "") {
- outTuple.f0 = line;
- collector.collect(outTuple);
+ collector.collect(line);
}
line = br.readLine();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
index edadfc3..9cfb2ce 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
@@ -23,28 +23,25 @@ import java.io.BufferedReader;
import java.io.FileReader;
import java.io.IOException;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
-public class FileStreamFunction extends SourceFunction<Tuple1<String>> {
+public class FileStreamFunction extends SourceFunction<String> {
private static final long serialVersionUID = 1L;
private final String path;
- private Tuple1<String> outTuple = new Tuple1<String>();
public FileStreamFunction(String path) {
this.path = path;
}
@Override
- public void invoke(Collector<Tuple1<String>> collector) throws IOException {
+ public void invoke(Collector<String> collector) throws IOException {
while (true) {
BufferedReader br = new BufferedReader(new FileReader(path));
String line = br.readLine();
while (line != null) {
if (line != "") {
- outTuple.f0 = line;
- collector.collect(outTuple);
+ collector.collect(line);
}
line = br.readLine();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
index dfe29d2..89f5182 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FromElementsFunction.java
@@ -22,14 +22,12 @@ package org.apache.flink.streaming.api.function.source;
import java.util.Arrays;
import java.util.Collection;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
-public class FromElementsFunction<T> extends SourceFunction<Tuple1<T>> {
+public class FromElementsFunction<T> extends SourceFunction<T> {
private static final long serialVersionUID = 1L;
Iterable<T> iterable;
- Tuple1<T> outTuple = new Tuple1<T>();
public FromElementsFunction(T... elements) {
this.iterable = Arrays.asList(elements);
@@ -40,10 +38,9 @@ public class FromElementsFunction<T> extends SourceFunction<Tuple1<T>> {
}
@Override
- public void invoke(Collector<Tuple1<T>> collector) throws Exception {
+ public void invoke(Collector<T> collector) throws Exception {
for (T element : iterable) {
- outTuple.f0 = element;
- collector.collect(outTuple);
+ collector.collect(element);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
index 706295e..d402374 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
@@ -26,7 +26,7 @@ import org.apache.flink.util.Collector;
* Source Function used to generate the number sequence
*
*/
-public class GenSequenceFunction extends SourceFunction<Tuple1<Long>> {
+public class GenSequenceFunction extends SourceFunction<Long> {
private static final long serialVersionUID = 1L;
@@ -40,10 +40,9 @@ public class GenSequenceFunction extends SourceFunction<Tuple1<Long>> {
}
@Override
- public void invoke(Collector<Tuple1<Long>> collector) throws Exception {
+ public void invoke(Collector<Long> collector) throws Exception {
for (long i = from; i <= to; i++) {
- outTuple.f0 = i;
- collector.collect(outTuple);
+ collector.collect(i);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
index 971533f..01d4dac 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
@@ -20,10 +20,9 @@
package org.apache.flink.streaming.api.function.source;
import org.apache.flink.api.common.functions.AbstractFunction;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.util.Collector;
-public abstract class SourceFunction<OUT extends Tuple> extends AbstractFunction {
+public abstract class SourceFunction<OUT> extends AbstractFunction {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index 81cfa81..92b1ea6 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -19,10 +19,9 @@
package org.apache.flink.streaming.api.invokable;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
-public class SinkInvokable<IN extends Tuple> extends StreamRecordInvokable<IN, IN> {
+public class SinkInvokable<IN> extends StreamRecordInvokable<IN, IN> {
private static final long serialVersionUID = 1L;
private SinkFunction<IN> sinkFunction;
@@ -33,16 +32,16 @@ public class SinkInvokable<IN extends Tuple> extends StreamRecordInvokable<IN, I
@Override
protected void immutableInvoke() throws Exception {
- while (recordIterator.next(reuse) != null) {
- sinkFunction.invoke((IN) reuse.getTuple());
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ sinkFunction.invoke((IN) reuse.getObject());
resetReuse();
}
}
@Override
protected void mutableInvoke() throws Exception {
- while (recordIterator.next(reuse) != null) {
- sinkFunction.invoke((IN) reuse.getTuple());
+ while ((reuse = recordIterator.next(reuse)) != null) {
+ sinkFunction.invoke((IN) reuse.getObject());
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
index 992a25e..c7f0f09 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
@@ -21,10 +21,9 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.function.source.SourceFunction;
-public class SourceInvokable<OUT extends Tuple> extends StreamComponentInvokable<OUT> implements
+public class SourceInvokable<OUT> extends StreamComponentInvokable<OUT> implements
Serializable {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
index daa7378..c011284 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
@@ -21,10 +21,9 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.util.Collector;
-public abstract class StreamComponentInvokable<OUT extends Tuple> implements Serializable {
+public abstract class StreamComponentInvokable<OUT> implements Serializable {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
index 5be3c30..b1cdde1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
@@ -21,13 +21,12 @@ package org.apache.flink.streaming.api.invokable;
import java.io.IOException;
-import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
-public abstract class StreamRecordInvokable<IN extends Tuple, OUT extends Tuple> extends
+public abstract class StreamRecordInvokable<IN, OUT> extends
StreamComponentInvokable<OUT> {
private static final long serialVersionUID = 1L;
[24/51] [abbrv] git commit: [streaming] Wrapped serializers to make
component construction simpler
Posted by se...@apache.org.
[streaming] Wrapped serializers to make component construction simpler
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/799424d1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/799424d1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/799424d1
Branch: refs/heads/master
Commit: 799424d1a3237a8d1b32451f4b3ad23b342cf59e
Parents: 2f704ae
Author: ghermann <re...@gmail.com>
Authored: Tue Jul 29 20:00:11 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:09 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 331 +++++++++++++---
.../streaming/api/IterativeDataStream.java | 21 +-
.../flink/streaming/api/StreamConfig.java | 87 +++--
.../api/StreamExecutionEnvironment.java | 380 ++-----------------
.../api/function/source/FileSourceFunction.java | 6 +-
.../api/function/source/FileStreamFunction.java | 10 +-
.../api/function/source/SourceFunction.java | 7 +-
.../api/invokable/SourceInvokable.java | 44 +++
.../api/invokable/StreamComponentInvokable.java | 10 +-
.../api/invokable/StreamRecordInvokable.java | 7 +-
.../api/invokable/UserSourceInvokable.java | 34 --
.../operator/StreamReduceInvokable.java | 1 -
.../api/invokable/operator/co/CoInvokable.java | 2 +-
.../AbstractStreamComponent.java | 78 ++--
.../streamcomponent/BlockingQueueBroker.java | 5 +-
.../api/streamcomponent/CoStreamTask.java | 36 +-
.../SingleInputAbstractStreamComponent.java | 50 +--
.../streamcomponent/StreamIterationSink.java | 8 +-
.../streamcomponent/StreamIterationSource.java | 6 +-
.../api/streamcomponent/StreamSink.java | 16 +-
.../api/streamcomponent/StreamSource.java | 17 +-
.../api/streamcomponent/StreamTask.java | 17 +-
.../util/serialization/FunctionTypeWrapper.java | 74 ++++
.../util/serialization/ObjectTypeWrapper.java | 61 +++
.../serialization/TypeSerializerWrapper.java | 57 +++
.../flink/streaming/api/WriteAsTextTest.java | 18 +-
.../api/invokable/operator/FilterTest.java | 2 +-
.../serialization/TypeSerializationTest.java | 91 +++++
28 files changed, 846 insertions(+), 630 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index d965bf2..1bde6a6 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -22,6 +22,9 @@ package org.apache.flink.streaming.api;
import java.util.ArrayList;
import java.util.List;
+import org.apache.commons.lang3.SerializationException;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.common.functions.AbstractFunction;
import org.apache.flink.api.java.functions.FilterFunction;
import org.apache.flink.api.java.functions.FlatMapFunction;
import org.apache.flink.api.java.functions.GroupReduceFunction;
@@ -29,14 +32,20 @@ import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
+import org.apache.flink.streaming.api.function.sink.PrintSinkFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
import org.apache.flink.streaming.api.function.sink.WriteFormatAsText;
+import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByBatches;
+import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByMillis;
+import org.apache.flink.streaming.api.invokable.SinkInvokable;
+import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
import org.apache.flink.streaming.api.invokable.operator.WindowReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
import org.apache.flink.streaming.partitioner.DistributePartitioner;
@@ -44,6 +53,8 @@ import org.apache.flink.streaming.partitioner.FieldsPartitioner;
import org.apache.flink.streaming.partitioner.ForwardPartitioner;
import org.apache.flink.streaming.partitioner.ShufflePartitioner;
import org.apache.flink.streaming.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
+import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
/**
* A DataStream represents a stream of elements of the same type. A DataStream
@@ -66,12 +77,13 @@ public class DataStream<T extends Tuple> {
protected String id;
protected int degreeOfParallelism;
protected String userDefinedName;
- protected OutputSelector<T> outputSelector;
protected List<String> connectIDs;
protected List<StreamPartitioner<T>> partitioners;
protected boolean iterationflag;
protected Integer iterationID;
+ protected JobGraphBuilder jobGraphBuilder;
+
/**
* Create a new {@link DataStream} in the given execution environment with
* partitioning set to shuffle by default.
@@ -91,8 +103,8 @@ public class DataStream<T extends Tuple> {
this.id = operatorType + "-" + counter.toString();
this.environment = environment;
this.degreeOfParallelism = environment.getDegreeOfParallelism();
+ this.jobGraphBuilder = environment.getJobGraphBuilder();
initConnections();
-
}
/**
@@ -106,11 +118,11 @@ public class DataStream<T extends Tuple> {
this.id = dataStream.id;
this.degreeOfParallelism = dataStream.degreeOfParallelism;
this.userDefinedName = dataStream.userDefinedName;
- this.outputSelector = dataStream.outputSelector;
this.connectIDs = new ArrayList<String>(dataStream.connectIDs);
this.partitioners = new ArrayList<StreamPartitioner<T>>(dataStream.partitioners);
this.iterationflag = dataStream.iterationflag;
this.iterationID = dataStream.iterationID;
+ this.jobGraphBuilder = dataStream.jobGraphBuilder;
}
/**
@@ -144,7 +156,7 @@ public class DataStream<T extends Tuple> {
* @return The DataStream with mutability set.
*/
public DataStream<T> setMutability(boolean isMutable) {
- environment.setMutability(this, isMutable);
+ jobGraphBuilder.setMutability(id, isMutable);
return this;
}
@@ -157,7 +169,7 @@ public class DataStream<T extends Tuple> {
* @return The DataStream with buffer timeout set.
*/
public DataStream<T> setBufferTimeout(long timeoutMillis) {
- environment.setBufferTimeout(this, timeoutMillis);
+ jobGraphBuilder.setBufferTimeout(id, timeoutMillis);
return this;
}
@@ -175,10 +187,9 @@ public class DataStream<T extends Tuple> {
}
this.degreeOfParallelism = dop;
- environment.setOperatorParallelism(this);
+ jobGraphBuilder.setParallelism(id, degreeOfParallelism);
return new DataStream<T>(this);
-
}
/**
@@ -200,13 +211,14 @@ public class DataStream<T extends Tuple> {
* @return The named DataStream.
*/
public DataStream<T> name(String name) {
- // copy?
+ // TODO copy DataStream?
if (name == "") {
throw new IllegalArgumentException("User defined name must not be empty string");
}
userDefinedName = name;
- environment.setName(this, name);
+ jobGraphBuilder.setUserDefinedName(id, name);
+
return this;
}
@@ -236,13 +248,10 @@ public class DataStream<T extends Tuple> {
* The other DataStream will connected to this
* @param stream
* This DataStream will be connected to returnStream
- * @return Connected DataStream
*/
- private DataStream<T> addConnection(DataStream<T> returnStream, DataStream<T> stream) {
+ private void addConnection(DataStream<T> returnStream, DataStream<T> stream) {
returnStream.connectIDs.addAll(stream.connectIDs);
returnStream.partitioners.addAll(stream.partitioners);
-
- return returnStream;
}
/**
@@ -256,8 +265,12 @@ public class DataStream<T extends Tuple> {
* @return The directed DataStream.
*/
public DataStream<T> directTo(OutputSelector<T> outputSelector) {
- this.outputSelector = outputSelector;
- environment.addDirectedEmit(id, outputSelector);
+ try {
+ jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize OutputSelector");
+ }
+
return this;
}
@@ -323,10 +336,10 @@ public class DataStream<T extends Tuple> {
for (int i = 0; i < returnStream.partitioners.size(); i++) {
returnStream.partitioners.set(i, partitioner);
}
-
+
return returnStream;
}
-
+
/**
* Applies a Map transformation on a {@link DataStream}. The transformation
* calls a {@link MapFunction} for each element of the DataStream. Each
@@ -340,9 +353,8 @@ public class DataStream<T extends Tuple> {
* @return The transformed DataStream.
*/
public <R extends Tuple> StreamOperator<T, R> map(MapFunction<T, R> mapper) {
- return environment.addFunction("map", new DataStream<T>(this), mapper,
- new MapInvokable<T, R>(mapper));
-
+ return addFunction("map", mapper, new FunctionTypeWrapper<T, Tuple, R>(mapper,
+ MapFunction.class, 0, -1, 1), new MapInvokable<T, R>(mapper));
}
/**
@@ -362,8 +374,10 @@ public class DataStream<T extends Tuple> {
*/
public <T2 extends Tuple, R extends Tuple> DataStream<R> coMapWith(
CoMapFunction<T, T2, R> coMapper, DataStream<T2> otherStream) {
- return environment.addCoFunction("coMap", new DataStream<T>(this), new DataStream<T2>(
- otherStream), coMapper, new CoMapInvokable<T, T2, R>(coMapper));
+ return addCoFunction("coMap", new DataStream<T>(this), new DataStream<T2>(otherStream),
+ coMapper,
+ new FunctionTypeWrapper<T, T2, R>(coMapper, CoMapFunction.class, 0, 1, 2),
+ new CoMapInvokable<T, T2, R>(coMapper));
}
/**
@@ -381,8 +395,8 @@ public class DataStream<T extends Tuple> {
* @return The transformed DataStream.
*/
public <R extends Tuple> StreamOperator<T, R> flatMap(FlatMapFunction<T, R> flatMapper) {
- return environment.addFunction("flatMap", new DataStream<T>(this), flatMapper,
- new FlatMapInvokable<T, R>(flatMapper));
+ return addFunction("flatMap", flatMapper, new FunctionTypeWrapper<T, Tuple, R>(flatMapper,
+ FlatMapFunction.class, 0, -1, 1), new FlatMapInvokable<T, R>(flatMapper));
}
/**
@@ -397,8 +411,8 @@ public class DataStream<T extends Tuple> {
* @return The filtered DataStream.
*/
public StreamOperator<T, T> filter(FilterFunction<T> filter) {
- return environment.addFunction("filter", new DataStream<T>(this), filter,
- new FilterInvokable<T>(filter));
+ return addFunction("filter", filter, new FunctionTypeWrapper<T, Tuple, T>(filter,
+ FilterFunction.class, 0, -1, 0), new FilterInvokable<T>(filter));
}
/**
@@ -418,8 +432,9 @@ public class DataStream<T extends Tuple> {
*/
public <R extends Tuple> StreamOperator<T, R> batchReduce(GroupReduceFunction<T, R> reducer,
int batchSize) {
- return environment.addFunction("batchReduce", new DataStream<T>(this), reducer,
- new BatchReduceInvokable<T, R>(reducer, batchSize));
+ return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
+ GroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<T, R>(reducer,
+ batchSize));
}
/**
@@ -440,8 +455,93 @@ public class DataStream<T extends Tuple> {
*/
public <R extends Tuple> StreamOperator<T, R> windowReduce(GroupReduceFunction<T, R> reducer,
long windowSize) {
- return environment.addFunction("batchReduce", new DataStream<T>(this), reducer,
- new WindowReduceInvokable<T, R>(reducer, windowSize));
+ return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
+ GroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<T, R>(reducer,
+ windowSize));
+ }
+
+ /**
+ * Internal function for passing the user defined functions to the JobGraph
+ * of the job.
+ *
+ * @param functionName
+ * name of the function
+ * @param function
+ * the user defined function
+ * @param functionInvokable
+ * the wrapping JobVertex instance
+ * @param <T>
+ * type of the input stream
+ * @param <R>
+ * type of the return stream
+ * @return the data stream constructed
+ */
+ private <R extends Tuple> StreamOperator<T, R> addFunction(String functionName,
+ final AbstractFunction function, TypeSerializerWrapper<T, Tuple, R> typeWrapper,
+ UserTaskInvokable<T, R> functionInvokable) {
+
+ DataStream<T> inputStream = new DataStream<T>(this);
+ StreamOperator<T, R> returnStream = new StreamOperator<T, R>(environment, functionName);
+
+ try {
+ jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, typeWrapper,
+ functionName, SerializationUtils.serialize(function), degreeOfParallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize user defined function");
+ }
+
+ connectGraph(inputStream, returnStream.getId(), 0);
+
+ if (inputStream.iterationflag) {
+ returnStream.addIterationSource(inputStream.iterationID.toString());
+ inputStream.iterationflag = false;
+ }
+
+ return returnStream;
+ }
+
+ protected <T1 extends Tuple, T2 extends Tuple, R extends Tuple> DataStream<R> addCoFunction(
+ String functionName, DataStream<T1> inputStream1, DataStream<T2> inputStream2,
+ final AbstractFunction function, TypeSerializerWrapper<T1, T2, R> typeWrapper,
+ CoInvokable<T1, T2, R> functionInvokable) {
+
+ DataStream<R> returnStream = new DataStream<R>(environment, functionName);
+
+ try {
+ jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, typeWrapper,
+ functionName, SerializationUtils.serialize(function), degreeOfParallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize user defined function");
+ }
+
+ connectGraph(inputStream1, returnStream.getId(), 1);
+ connectGraph(inputStream2, returnStream.getId(), 2);
+
+ // TODO consider iteration
+
+ return returnStream;
+ }
+
+ /**
+ * Internal function for assembling the underlying
+ * {@link org.apache.flink.nephele.jobgraph.JobGraph} of the job. Connects
+ * the outputs of the given input stream to the specified output stream
+ * given by the outputID.
+ *
+ * @param inputStream
+ * input data stream
+ * @param outputID
+ * ID of the output
+ * @param typeNumber
+ * Number of the type (used at co-functions)
+ */
+ <X extends Tuple> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
+ for (int i = 0; i < inputStream.connectIDs.size(); i++) {
+ String inputID = inputStream.connectIDs.get(i);
+ StreamPartitioner<X> partitioner = inputStream.partitioners.get(i);
+
+ jobGraphBuilder.setEdge(inputID, outputID, partitioner, typeNumber);
+ }
}
/**
@@ -454,7 +554,7 @@ public class DataStream<T extends Tuple> {
* @return The modified DataStream.
*/
public DataStream<T> addSink(SinkFunction<T> sinkFunction) {
- return environment.addSink(new DataStream<T>(this), sinkFunction);
+ return addSink(new DataStream<T>(this), sinkFunction);
}
/**
@@ -465,7 +565,35 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream.
*/
public DataStream<T> print() {
- return environment.print(new DataStream<T>(this));
+ DataStream<T> inputStream = new DataStream<T>(this);
+ PrintSinkFunction<T> printFunction = new PrintSinkFunction<T>();
+ DataStream<T> returnStream = addSink(inputStream, printFunction, null);
+
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+
+ return returnStream;
+ }
+
+ private DataStream<T> addSink(DataStream<T> inputStream, SinkFunction<T> sinkFunction) {
+ return addSink(inputStream, sinkFunction, new FunctionTypeWrapper<T, Tuple, T>(
+ sinkFunction, SinkFunction.class, 0, -1, 0));
+ }
+
+ private DataStream<T> addSink(DataStream<T> inputStream, SinkFunction<T> sinkFunction,
+ TypeSerializerWrapper<T, Tuple, T> typeWrapper) {
+ DataStream<T> returnStream = new DataStream<T>(environment, "sink");
+
+ try {
+ jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<T>(sinkFunction),
+ typeWrapper, "sink", SerializationUtils.serialize(sinkFunction),
+ degreeOfParallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize SinkFunction");
+ }
+
+ inputStream.connectGraph(inputStream, returnStream.getId(), 0);
+
+ return returnStream;
}
/**
@@ -479,7 +607,7 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path) {
- environment.writeAsText(this, path, new WriteFormatAsText<T>(), 1, null);
+ writeAsText(this, path, new WriteFormatAsText<T>(), 1, null);
return new DataStream<T>(this);
}
@@ -497,7 +625,7 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path, long millis) {
- environment.writeAsText(this, path, new WriteFormatAsText<T>(), millis, null);
+ writeAsText(this, path, new WriteFormatAsText<T>(), millis, null);
return new DataStream<T>(this);
}
@@ -516,7 +644,7 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path, int batchSize) {
- environment.writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, null);
+ writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, null);
return new DataStream<T>(this);
}
@@ -539,7 +667,7 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path, long millis, T endTuple) {
- environment.writeAsText(this, path, new WriteFormatAsText<T>(), millis, endTuple);
+ writeAsText(this, path, new WriteFormatAsText<T>(), millis, endTuple);
return new DataStream<T>(this);
}
@@ -563,11 +691,66 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsText(String path, int batchSize, T endTuple) {
- environment.writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, endTuple);
+ writeAsText(this, path, new WriteFormatAsText<T>(), batchSize, endTuple);
return new DataStream<T>(this);
}
/**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically, in every millis milliseconds. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param millis
+ * is the file update frequency
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * system time.
+ *
+ * @return the data stream constructed
+ */
+ private DataStream<T> writeAsText(DataStream<T> inputStream, String path,
+ WriteFormatAsText<T> format, long millis, T endTuple) {
+ DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<T>(path,
+ format, millis, endTuple), null);
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
+ return returnStream;
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in text format. The
+ * writing is performed periodically in equally sized batches. For every
+ * element of the DataStream the result of {@link Object#toString()} is
+ * written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param batchSize
+ * is the size of the batches, i.e. the number of tuples written
+ * to the file at a time
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * batchSize.
+ *
+ * @return the data stream constructed
+ */
+ private DataStream<T> writeAsText(DataStream<T> inputStream, String path,
+ WriteFormatAsText<T> format, int batchSize, T endTuple) {
+ DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<T>(path,
+ format, batchSize, endTuple), null);
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
+ return returnStream;
+ }
+
+ /**
* Writes a DataStream to the file specified by path in text format. For
* every element of the DataStream the result of {@link Object#toString()}
* is written.
@@ -578,7 +761,7 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsCsv(String path) {
- environment.writeAsCsv(this, path, new WriteFormatAsCsv<T>(), 1, null);
+ writeAsCsv(this, path, new WriteFormatAsCsv<T>(), 1, null);
return new DataStream<T>(this);
}
@@ -596,7 +779,7 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsCsv(String path, long millis) {
- environment.writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, null);
+ writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, null);
return new DataStream<T>(this);
}
@@ -615,7 +798,7 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsCsv(String path, int batchSize) {
- environment.writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, null);
+ writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, null);
return new DataStream<T>(this);
}
@@ -638,7 +821,7 @@ public class DataStream<T extends Tuple> {
* @return The closed DataStream
*/
public DataStream<T> writeAsCsv(String path, long millis, T endTuple) {
- environment.writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, endTuple);
+ writeAsCsv(this, path, new WriteFormatAsCsv<T>(), millis, endTuple);
return new DataStream<T>(this);
}
@@ -663,11 +846,66 @@ public class DataStream<T extends Tuple> {
*/
public DataStream<T> writeAsCsv(String path, int batchSize, T endTuple) {
setMutability(false);
- environment.writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, endTuple);
+ writeAsCsv(this, path, new WriteFormatAsCsv<T>(), batchSize, endTuple);
return new DataStream<T>(this);
}
/**
+ * Writes a DataStream to the file specified by path in csv format. The
+ * writing is performed periodically, in every millis milliseconds. For
+ * every element of the DataStream the result of {@link Object#toString()}
+ * is written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param millis
+ * is the file update frequency
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * system time.
+ *
+ * @return the data stream constructed
+ */
+ private DataStream<T> writeAsCsv(DataStream<T> inputStream, String path,
+ WriteFormatAsCsv<T> format, long millis, T endTuple) {
+ DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<T>(path,
+ format, millis, endTuple));
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
+ return returnStream;
+ }
+
+ /**
+ * Writes a DataStream to the file specified by path in csv format. The
+ * writing is performed periodically in equally sized batches. For every
+ * element of the DataStream the result of {@link Object#toString()} is
+ * written.
+ *
+ * @param path
+ * is the path to the location where the tuples are written
+ * @param batchSize
+ * is the size of the batches, i.e. the number of tuples written
+ * to the file at a time
+ * @param endTuple
+ * is a special tuple indicating the end of the stream. If an
+ * endTuple is caught, the last pending batch of tuples will be
+ * immediately appended to the target file regardless of the
+ * batchSize.
+ *
+ * @return the data stream constructed
+ */
+ private DataStream<T> writeAsCsv(DataStream<T> inputStream, String path,
+ WriteFormatAsCsv<T> format, int batchSize, T endTuple) {
+ DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<T>(path,
+ format, batchSize, endTuple), null);
+ jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+ jobGraphBuilder.setMutability(returnStream.getId(), false);
+ return returnStream;
+ }
+
+ /**
* Initiates an iterative part of the program that executes multiple times
* and feeds back data streams. The iterative part needs to be closed by
* calling {@link IterativeDataStream#closeWith(DataStream)}. The data
@@ -688,9 +926,12 @@ public class DataStream<T extends Tuple> {
return new IterativeDataStream<T>(this);
}
- protected DataStream<T> addIterationSource(String iterationID) {
- environment.addIterationSource(this, iterationID);
+ protected <R extends Tuple> DataStream<T> addIterationSource(String iterationID) {
+ DataStream<R> returnStream = new DataStream<R>(environment, "iterationSource");
+
+ jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
+ degreeOfParallelism);
+
return new DataStream<T>(this);
}
-
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
index c179f83..1cfb625 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
@@ -20,6 +20,7 @@
package org.apache.flink.streaming.api;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.partitioner.ForwardPartitioner;
/**
* The iterative data stream represents the start of an iteration in a
@@ -63,14 +64,26 @@ public class IterativeDataStream<T extends Tuple> extends StreamOperator<T, T> {
*
* @param iterationResult
* The data stream that can be fed back to the next iteration.
- * @param directName
+ * @param iterationName
* Name of the iteration edge (backward edge to iteration head)
* when used with directed emits
*
*/
- public DataStream<T> closeWith(DataStream<T> iterationResult, String directName) {
- environment.addIterationSink(iterationResult, iterationID.toString(), directName);
+ public <R extends Tuple> DataStream<T> closeWith(DataStream<T> iterationResult,
+ String iterationName) {
+ DataStream<R> returnStream = new DataStream<R>(environment, "iterationSink");
+
+ jobGraphBuilder.addIterationSink(returnStream.getId(), iterationResult.getId(),
+ iterationID.toString(), iterationResult.getParallelism(), iterationName);
+
+ jobGraphBuilder.setIterationSourceParallelism(iterationID.toString(),
+ iterationResult.getParallelism());
+
+ for (int i = 0; i < iterationResult.connectIDs.size(); i++) {
+ String inputID = iterationResult.connectIDs.get(i);
+ jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<T>(), 0);
+ }
+
return iterationResult;
}
-
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index fc4a1dd..3d49928 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -23,6 +23,7 @@ import java.io.IOException;
import org.apache.commons.lang3.SerializationException;
import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.common.functions.AbstractFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.collector.OutputSelector;
@@ -30,6 +31,7 @@ import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
import org.apache.flink.streaming.api.streamcomponent.StreamComponentException;
import org.apache.flink.streaming.partitioner.ShufflePartitioner;
import org.apache.flink.streaming.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
public class StreamConfig {
private static final String INPUT_TYPE = "inputType_";
@@ -71,6 +73,23 @@ public class StreamConfig {
// CONFIGS
+ public void setTypeWrapper(
+ TypeSerializerWrapper<? extends Tuple, ? extends Tuple, ? extends Tuple> typeWrapper) {
+ config.setBytes("typeWrapper", SerializationUtils.serialize(typeWrapper));
+ }
+
+ @SuppressWarnings("unchecked")
+ public <IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> TypeSerializerWrapper<IN1, IN2, OUT> getTypeWrapper() {
+ byte[] serializedWrapper = config.getBytes("typeWrapper", null);
+
+ if (serializedWrapper == null) {
+ throw new RuntimeException("TypeSerializationWrapper must be set");
+ }
+
+ return (TypeSerializerWrapper<IN1, IN2, OUT>) SerializationUtils
+ .deserialize(serializedWrapper);
+ }
+
public void setMutability(boolean isMutable) {
config.setBoolean(MUTABILITY, isMutable);
}
@@ -87,25 +106,26 @@ public class StreamConfig {
return config.getLong(BUFFER_TIMEOUT, DEFAULT_TIMEOUT);
}
- public void setUserInvokableClass(Class<? extends StreamComponentInvokable> clazz) {
- config.setClass(USER_FUNCTION, clazz);
- }
+ public void setUserInvokable(StreamComponentInvokable<? extends Tuple> invokableObject) {
+ if (invokableObject != null) {
+ config.setClass(USER_FUNCTION, invokableObject.getClass());
- @SuppressWarnings("unchecked")
- public <T extends StreamComponentInvokable> Class<? extends T> getUserInvokableClass() {
- return (Class<? extends T>) config.getClass(USER_FUNCTION, null);
- }
-
- public void setUserInvokableObject(StreamComponentInvokable invokableObject) {
- try {
- config.setBytes(SERIALIZEDUDF, SerializationUtils.serialize(invokableObject));
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize invokable object "
- + invokableObject.getClass(), e);
+ try {
+ config.setBytes(SERIALIZEDUDF, SerializationUtils.serialize(invokableObject));
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize invokable object "
+ + invokableObject.getClass(), e);
+ }
}
}
- public <T extends StreamComponentInvokable> T getUserInvokableObject() {
+ // @SuppressWarnings("unchecked")
+ // public <T extends StreamComponentInvokable> Class<? extends T>
+ // getUserInvokableClass() {
+ // return (Class<? extends T>) config.getClass(USER_FUNCTION, null);
+ // }
+
+ public <T extends Tuple> StreamComponentInvokable<T> getUserInvokableObject() {
try {
return deserializeObject(config.getBytes(SERIALIZEDUDF, null));
} catch (Exception e) {
@@ -122,27 +142,29 @@ public class StreamConfig {
return config.getString(COMPONENT_NAME, null);
}
- public void setFunction(byte[] serializedFunction) {
- config.setBytes(FUNCTION, serializedFunction);
+ public void setFunction(byte[] serializedFunction, String functionName) {
+ if (serializedFunction != null) {
+ config.setBytes(FUNCTION, serializedFunction);
+ config.setString(FUNCTION_NAME, functionName);
+ }
}
public Object getFunction() {
try {
- return SerializationUtils.deserialize(config
- .getBytes(FUNCTION, null));
+ return SerializationUtils.deserialize(config.getBytes(FUNCTION, null));
} catch (SerializationException e) {
throw new RuntimeException("Cannot deserialize invokable object", e);
}
}
- public void setFunctionName(String functionName) {
- config.setString(FUNCTION_NAME, functionName);
- }
+ // public void setFunctionName(String functionName) {
+ // config.setString(FUNCTION_NAME, functionName);
+ // }
public String getFunctionName() {
return config.getString(FUNCTION_NAME, "");
}
-
+
public void setUserDefinedName(String userDefinedName) {
if (userDefinedName != null) {
config.setString(USER_DEFINED_NAME, userDefinedName);
@@ -158,8 +180,10 @@ public class StreamConfig {
}
public void setOutputSelector(byte[] outputSelector) {
- config.setBytes(OUTPUT_SELECTOR, outputSelector);
-
+ if (outputSelector != null) {
+ setDirectedEmit(true);
+ config.setBytes(OUTPUT_SELECTOR, outputSelector);
+ }
}
public <T extends Tuple> OutputSelector<T> getOutputSelector() {
@@ -174,7 +198,7 @@ public class StreamConfig {
public void setIterationId(String iterationId) {
config.setString(ITERATION_ID, iterationId);
}
-
+
public String getIterationId() {
return config.getString(ITERATION_ID, "iteration-0");
}
@@ -233,7 +257,16 @@ public class StreamConfig {
public int getInputType(int inputNumber) {
return config.getInteger(INPUT_TYPE + inputNumber, 0);
}
-
+
+ public void setFunctionClass(Class<? extends AbstractFunction> functionClass) {
+ config.setClass("functionClass", functionClass);
+ }
+
+ @SuppressWarnings("unchecked")
+ public Class<? extends AbstractFunction> getFunctionClass() {
+ return (Class<? extends AbstractFunction>) config.getClass("functionClass", null);
+ }
+
@SuppressWarnings("unchecked")
protected static <T> T deserializeObject(byte[] serializedObject) throws IOException,
ClassNotFoundException {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index f56614d..4539126 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -24,27 +24,17 @@ import java.util.Collection;
import org.apache.commons.lang3.SerializationException;
import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.common.functions.AbstractFunction;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.collector.OutputSelector;
-import org.apache.flink.streaming.api.function.sink.PrintSinkFunction;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
-import org.apache.flink.streaming.api.function.sink.WriteFormatAsText;
-import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByBatches;
-import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByMillis;
import org.apache.flink.streaming.api.function.source.FileSourceFunction;
import org.apache.flink.streaming.api.function.source.FileStreamFunction;
import org.apache.flink.streaming.api.function.source.FromElementsFunction;
import org.apache.flink.streaming.api.function.source.GenSequenceFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.streaming.api.invokable.SinkInvokable;
-import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
-import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
-import org.apache.flink.streaming.partitioner.ForwardPartitioner;
-import org.apache.flink.streaming.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.api.invokable.SourceInvokable;
+import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
+import org.apache.flink.streaming.util.serialization.ObjectTypeWrapper;
/**
* {@link ExecutionEnvironment} for streaming jobs. An instance of it is
@@ -124,13 +114,14 @@ public abstract class StreamExecutionEnvironment {
this.degreeOfParallelism = degreeOfParallelism;
}
- protected void setMutability(DataStream<?> stream, boolean isMutable) {
- jobGraphBuilder.setMutability(stream.getId(), isMutable);
- }
-
- protected void setBufferTimeout(DataStream<?> stream, long bufferTimeout) {
- jobGraphBuilder.setBufferTimeout(stream.getId(), bufferTimeout);
- }
+ // protected void setMutability(DataStream<?> stream, boolean isMutable) {
+ // jobGraphBuilder.setMutability(stream.getId(), isMutable);
+ // }
+ //
+ // protected void setBufferTimeout(DataStream<?> stream, long bufferTimeout)
+ // {
+ // jobGraphBuilder.setBufferTimeout(stream.getId(), bufferTimeout);
+ // }
/**
* Sets the number of hardware contexts (CPU cores / threads) used when
@@ -204,8 +195,11 @@ public abstract class StreamExecutionEnvironment {
DataStream<Tuple1<X>> returnStream = new DataStream<Tuple1<X>>(this, "elements");
try {
- jobGraphBuilder.addSource(returnStream.getId(), new FromElementsFunction<X>(data),
- "elements", SerializationUtils.serialize(data[0]), 1);
+ SourceFunction<Tuple1<X>> function = new FromElementsFunction<X>(data);
+ jobGraphBuilder.addSource(returnStream.getId(),
+ new SourceInvokable<Tuple1<X>>(function),
+ new ObjectTypeWrapper<Tuple1<X>, Tuple, Tuple1<X>>(data[0], null, data[0]),
+ "source", SerializationUtils.serialize(function), 1);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize elements");
}
@@ -232,8 +226,14 @@ public abstract class StreamExecutionEnvironment {
}
try {
- jobGraphBuilder.addSource(returnStream.getId(), new FromElementsFunction<X>(data),
- "elements", SerializationUtils.serialize((Serializable) data.toArray()[0]), 1);
+ SourceFunction<Tuple1<X>> function = new FromElementsFunction<X>(data);
+
+ jobGraphBuilder
+ .addSource(returnStream.getId(), new SourceInvokable<Tuple1<X>>(
+ new FromElementsFunction<X>(data)),
+ new ObjectTypeWrapper<Tuple1<X>, Tuple, Tuple1<X>>(data.toArray()[0],
+ null, data.toArray()[0]), "source", SerializationUtils
+ .serialize(function), 1);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize collection");
}
@@ -257,7 +257,7 @@ public abstract class StreamExecutionEnvironment {
/**
* Ads a data source thus opening a {@link DataStream}.
*
- * @param sourceFunction
+ * @param function
* the user defined function
* @param parallelism
* number of parallel instances of the function
@@ -265,13 +265,13 @@ public abstract class StreamExecutionEnvironment {
* type of the returned stream
* @return the data stream constructed
*/
- public <T extends Tuple> DataStream<T> addSource(SourceFunction<T> sourceFunction,
- int parallelism) {
+ public <T extends Tuple> DataStream<T> addSource(SourceFunction<T> function, int parallelism) {
DataStream<T> returnStream = new DataStream<T>(this, "source");
try {
- jobGraphBuilder.addSource(returnStream.getId(), sourceFunction, "source",
- SerializationUtils.serialize(sourceFunction), parallelism);
+ jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<T>(function),
+ new FunctionTypeWrapper<T, Tuple, T>(function, SourceFunction.class, 0, -1, 0),
+ "source", SerializationUtils.serialize(function), parallelism);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize SourceFunction");
}
@@ -284,326 +284,6 @@ public abstract class StreamExecutionEnvironment {
}
// --------------------------------------------------------------------------------------------
- // Data stream operators and sinks
- // --------------------------------------------------------------------------------------------
-
- /**
- * Internal function for passing the user defined functions to the JobGraph
- * of the job.
- *
- * @param functionName
- * name of the function
- * @param inputStream
- * input data stream
- * @param function
- * the user defined function
- * @param functionInvokable
- * the wrapping JobVertex instance
- * @param <T>
- * type of the input stream
- * @param <R>
- * type of the return stream
- * @return the data stream constructed
- */
- protected <T extends Tuple, R extends Tuple> StreamOperator<T, R> addFunction(
- String functionName, DataStream<T> inputStream, final AbstractFunction function,
- UserTaskInvokable<T, R> functionInvokable) {
- StreamOperator<T, R> returnStream = new StreamOperator<T, R>(this, functionName);
-
- try {
- jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, functionName,
- SerializationUtils.serialize(function), degreeOfParallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize user defined function");
- }
-
- connectGraph(inputStream, returnStream.getId(), 0);
-
- if (inputStream.iterationflag) {
- returnStream.addIterationSource(inputStream.iterationID.toString());
- inputStream.iterationflag = false;
- }
-
- return returnStream;
- }
-
- protected <T1 extends Tuple, T2 extends Tuple, R extends Tuple> DataStream<R> addCoFunction(String functionName, DataStream<T1> inputStream1, DataStream<T2> inputStream2, final AbstractFunction function,
- CoInvokable<T1, T2, R> functionInvokable) {
-
- DataStream<R> returnStream = new DataStream<R>(this, functionName);
-
- try {
- jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, functionName,
- SerializationUtils.serialize(function), degreeOfParallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize user defined function");
- }
-
- connectGraph(inputStream1, returnStream.getId(), 1);
- connectGraph(inputStream2, returnStream.getId(), 2);
-
- // TODO consider iteration
-// if (inputStream.iterationflag) {
-// returnStream.addIterationSource(inputStream.iterationID.toString());
-// inputStream.iterationflag = false;
-// }
-
- return returnStream;
- }
-
- protected <T extends Tuple, R extends Tuple> void addIterationSource(DataStream<T> inputStream,
- String iterationID) {
- DataStream<R> returnStream = new DataStream<R>(this, "iterationSource");
-
- jobGraphBuilder.addIterationSource(returnStream.getId(), inputStream.getId(), iterationID,
- degreeOfParallelism);
-
- }
-
- protected <T extends Tuple, R extends Tuple> void addIterationSink(DataStream<T> inputStream,
- String iterationID, String iterationName) {
- DataStream<R> returnStream = new DataStream<R>(this, "iterationSink");
-
- jobGraphBuilder.addIterationSink(returnStream.getId(), inputStream.getId(), iterationID,
- inputStream.getParallelism(), iterationName);
-
- jobGraphBuilder.setIterationSourceParallelism(iterationID, inputStream.getParallelism());
-
- for (int i = 0; i < inputStream.connectIDs.size(); i++) {
- String inputID = inputStream.connectIDs.get(i);
- jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<T>(), 0);
- }
- }
-
- /**
- * Adds the given sink to this environment. Only streams with sinks added
- * will be executed once the {@link #execute()} method is called.
- *
- * @param inputStream
- * input data stream
- * @param sinkFunction
- * the user defined function
- * @param <T>
- * type of the returned stream
- * @return the data stream constructed
- */
- protected <T extends Tuple> DataStream<T> addSink(DataStream<T> inputStream,
- SinkFunction<T> sinkFunction) {
- DataStream<T> returnStream = new DataStream<T>(this, "sink");
-
- try {
- jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<T>(sinkFunction),
- "sink", SerializationUtils.serialize(sinkFunction), degreeOfParallelism);
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize SinkFunction");
- }
-
- connectGraph(inputStream, returnStream.getId(), 0);
-
- return returnStream;
- }
-
- <T extends Tuple> void addDirectedEmit(String id, OutputSelector<T> outputSelector) {
- try {
- jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
- } catch (SerializationException e) {
- throw new RuntimeException("Cannot serialize OutputSelector");
- }
- }
-
- /**
- * Writes a DataStream to the standard output stream (stdout). For each
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @param inputStream
- * the input data stream
- *
- * @param <T>
- * type of the returned stream
- * @return the data stream constructed
- */
- protected <T extends Tuple> DataStream<T> print(DataStream<T> inputStream) {
- DataStream<T> returnStream = addSink(inputStream, new PrintSinkFunction<T>());
-
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
-
- return returnStream;
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically, in every millis milliseconds. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param millis
- * is the file update frequency
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * system time.
- *
- * @return the data stream constructed
- */
- protected <T extends Tuple> DataStream<T> writeAsText(DataStream<T> inputStream, String path,
- WriteFormatAsText<T> format, long millis, T endTuple) {
- DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<T>(path,
- format, millis, endTuple));
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
- jobGraphBuilder.setMutability(returnStream.getId(), false);
- return returnStream;
- }
-
- /**
- * Writes a DataStream to the file specified by path in text format. The
- * writing is performed periodically in equally sized batches. For every
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param batchSize
- * is the size of the batches, i.e. the number of tuples written
- * to the file at a time
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * batchSize.
- *
- * @return the data stream constructed
- */
- protected <T extends Tuple> DataStream<T> writeAsText(DataStream<T> inputStream, String path,
- WriteFormatAsText<T> format, int batchSize, T endTuple) {
- DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<T>(path,
- format, batchSize, endTuple));
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
- jobGraphBuilder.setMutability(returnStream.getId(), false);
- return returnStream;
- }
-
- /**
- * Writes a DataStream to the file specified by path in csv format. The
- * writing is performed periodically, in every millis milliseconds. For
- * every element of the DataStream the result of {@link Object#toString()}
- * is written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param millis
- * is the file update frequency
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * system time.
- *
- * @return the data stream constructed
- */
- protected <T extends Tuple> DataStream<T> writeAsCsv(DataStream<T> inputStream, String path,
- WriteFormatAsCsv<T> format, long millis, T endTuple) {
- DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<T>(path,
- format, millis, endTuple));
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
- jobGraphBuilder.setMutability(returnStream.getId(), false);
- return returnStream;
- }
-
- /**
- * Writes a DataStream to the file specified by path in csv format. The
- * writing is performed periodically in equally sized batches. For every
- * element of the DataStream the result of {@link Object#toString()} is
- * written.
- *
- * @param path
- * is the path to the location where the tuples are written
- * @param batchSize
- * is the size of the batches, i.e. the number of tuples written
- * to the file at a time
- * @param endTuple
- * is a special tuple indicating the end of the stream. If an
- * endTuple is caught, the last pending batch of tuples will be
- * immediately appended to the target file regardless of the
- * batchSize.
- *
- * @return the data stream constructed
- */
- protected <T extends Tuple> DataStream<T> writeAsCsv(DataStream<T> inputStream, String path,
- WriteFormatAsCsv<T> format, int batchSize, T endTuple) {
- DataStream<T> returnStream = addSink(inputStream, new WriteSinkFunctionByBatches<T>(path,
- format, batchSize, endTuple));
- jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
- jobGraphBuilder.setMutability(returnStream.getId(), false);
- return returnStream;
- }
-
- /**
- * Internal function for assembling the underlying
- * {@link org.apache.flink.nephele.jobgraph.JobGraph} of the job. Connects
- * the outputs of the given input stream to the specified output stream
- * given by the outputID.
- *
- * @param inputStream
- * input data stream
- * @param outputID
- * ID of the output
- * @param <T>
- * type of the input stream
- * @param typeNumber
- * Number of the type (used at co-functions)
- */
- private <T extends Tuple> void connectGraph(DataStream<T> inputStream, String outputID, int typeNumber) {
-
- for (int i = 0; i < inputStream.connectIDs.size(); i++) {
- String inputID = inputStream.connectIDs.get(i);
- StreamPartitioner<T> partitioner = inputStream.partitioners.get(i);
-
- jobGraphBuilder.setEdge(inputID, outputID, partitioner,
- typeNumber);
- }
- }
-
- protected <T extends Tuple> void setName(DataStream<T> stream, String name) {
- jobGraphBuilder.setUserDefinedName(stream.getId(), name);
- }
-
- /**
- * Sets the proper parallelism for the given operator in the JobGraph
- *
- * @param inputStream
- * DataStream corresponding to the operator
- * @param <T>
- * type of the operator
- */
- protected <T extends Tuple> void setOperatorParallelism(DataStream<T> inputStream) {
- jobGraphBuilder.setParallelism(inputStream.getId(), inputStream.degreeOfParallelism);
- }
-
- // /**
- // * Converts object to byte array using default java serialization
- // *
- // * @param object
- // * Object to be serialized
- // * @return Serialized object
- // */
- // static byte[] serializeToByteArray(Serializable object) {
- // SerializationUtils.serialize(object);
- // ByteArrayOutputStream baos = new ByteArrayOutputStream();
- // ObjectOutputStream oos;
- // try {
- // oos = new ObjectOutputStream(baos);
- // oos.writeObject(object);
- // } catch (IOException e) {
- // throw new RuntimeException("Cannot serialize object: " + object);
- // }
- // return baos.toByteArray();
- // }
-
- // --------------------------------------------------------------------------------------------
// Instantiation of Execution Contexts
// --------------------------------------------------------------------------------------------
@@ -741,7 +421,7 @@ public abstract class StreamExecutionEnvironment {
*
* @return jobgraph
*/
- public JobGraphBuilder jobGB() {
+ public JobGraphBuilder getJobGraphBuilder() {
return jobGraphBuilder;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
index f016cbc..f6c2c72 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java
@@ -28,14 +28,14 @@ import org.apache.flink.util.Collector;
public class FileSourceFunction extends SourceFunction<Tuple1<String>> {
private static final long serialVersionUID = 1L;
-
+
private final String path;
private Tuple1<String> outTuple = new Tuple1<String>();
-
+
public FileSourceFunction(String path) {
this.path = path;
}
-
+
@Override
public void invoke(Collector<Tuple1<String>> collector) throws IOException {
BufferedReader br = new BufferedReader(new FileReader(path));
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
index c3ccedf..edadfc3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileStreamFunction.java
@@ -26,19 +26,19 @@ import java.io.IOException;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
-public class FileStreamFunction extends SourceFunction<Tuple1<String>>{
+public class FileStreamFunction extends SourceFunction<Tuple1<String>> {
private static final long serialVersionUID = 1L;
-
+
private final String path;
private Tuple1<String> outTuple = new Tuple1<String>();
-
+
public FileStreamFunction(String path) {
this.path = path;
}
-
+
@Override
public void invoke(Collector<Tuple1<String>> collector) throws IOException {
- while(true){
+ while (true) {
BufferedReader br = new BufferedReader(new FileReader(path));
String line = br.readLine();
while (line != null) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
index 70553bf..971533f 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
@@ -19,12 +19,13 @@
package org.apache.flink.streaming.api.function.source;
-import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
-
+import org.apache.flink.api.common.functions.AbstractFunction;
import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.util.Collector;
-public abstract class SourceFunction<OUT extends Tuple> extends UserSourceInvokable<OUT> {
+public abstract class SourceFunction<OUT extends Tuple> extends AbstractFunction {
private static final long serialVersionUID = 1L;
+ public abstract void invoke(Collector<OUT> collector) throws Exception;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
new file mode 100644
index 0000000..992a25e
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
@@ -0,0 +1,44 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
+
+public class SourceInvokable<OUT extends Tuple> extends StreamComponentInvokable<OUT> implements
+ Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private SourceFunction<OUT> sourceFunction;
+
+ public SourceInvokable() {
+ }
+
+ public SourceInvokable(SourceFunction<OUT> sourceFunction) {
+ this.sourceFunction = sourceFunction;
+ }
+
+ public void invoke() throws Exception {
+ sourceFunction.invoke(collector);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
index 0e8ea98..daa7378 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
@@ -21,7 +21,10 @@ package org.apache.flink.streaming.api.invokable;
import java.io.Serializable;
-public abstract class StreamComponentInvokable implements Serializable {
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.util.Collector;
+
+public abstract class StreamComponentInvokable<OUT extends Tuple> implements Serializable {
private static final long serialVersionUID = 1L;
@@ -29,6 +32,11 @@ public abstract class StreamComponentInvokable implements Serializable {
private String componentName;
@SuppressWarnings("unused")
private int channelID;
+ protected Collector<OUT> collector;
+
+ public void setCollector(Collector<OUT> collector) {
+ this.collector = collector;
+ }
public void setAttributes(String componentName, int channelID) {
this.componentName = componentName;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
index 6beec27..5be3c30 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
@@ -28,11 +28,10 @@ import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
public abstract class StreamRecordInvokable<IN extends Tuple, OUT extends Tuple> extends
- StreamComponentInvokable {
+ StreamComponentInvokable<OUT> {
private static final long serialVersionUID = 1L;
- protected Collector<OUT> collector;
protected MutableObjectIterator<StreamRecord<IN>> recordIterator;
StreamRecordSerializer<IN> serializer;
protected StreamRecord<IN> reuse;
@@ -41,7 +40,7 @@ public abstract class StreamRecordInvokable<IN extends Tuple, OUT extends Tuple>
public void initialize(Collector<OUT> collector,
MutableObjectIterator<StreamRecord<IN>> recordIterator,
StreamRecordSerializer<IN> serializer, boolean isMutable) {
- this.collector = collector;
+ setCollector(collector);
this.recordIterator = recordIterator;
this.serializer = serializer;
this.reuse = serializer.createInstance();
@@ -51,7 +50,7 @@ public abstract class StreamRecordInvokable<IN extends Tuple, OUT extends Tuple>
protected void resetReuse() {
this.reuse = serializer.createInstance();
}
-
+
protected StreamRecord<IN> loadNextRecord() {
try {
reuse = recordIterator.next(reuse);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserSourceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserSourceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserSourceInvokable.java
deleted file mode 100644
index e85b563..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/UserSourceInvokable.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.streaming.api.invokable;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.util.Collector;
-
-public abstract class UserSourceInvokable<OUT extends Tuple> extends StreamComponentInvokable implements
- Serializable {
-
- private static final long serialVersionUID = 1L;
-
- public abstract void invoke(Collector<OUT> collector) throws Exception;
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
index e881d57..f7ea566 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
@@ -28,5 +28,4 @@ public abstract class StreamReduceInvokable<IN extends Tuple, OUT extends Tuple>
private static final long serialVersionUID = 1L;
protected GroupReduceFunction<IN, OUT> reducer;
-
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
index 85086f9..884e361 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
@@ -27,7 +27,7 @@ import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
public abstract class CoInvokable<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
- StreamComponentInvokable {
+ StreamComponentInvokable<OUT> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index 22c079c..fcf87e2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -26,13 +26,8 @@ import org.apache.commons.lang.SerializationUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.FilterFunction;
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.functions.GroupReduceFunction;
-import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
@@ -43,17 +38,18 @@ import org.apache.flink.streaming.api.StreamConfig;
import org.apache.flink.streaming.api.collector.DirectedStreamCollector;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.collector.StreamCollector;
-import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
-import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
public abstract class AbstractStreamComponent<OUT extends Tuple> extends AbstractInvokable {
+ protected static final String SOURCE = "source";
+
private static final Log LOG = LogFactory.getLog(AbstractStreamComponent.class);
protected TupleTypeInfo<OUT> outTupleTypeInfo = null;
@@ -61,6 +57,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
protected SerializationDelegate<StreamRecord<OUT>> outSerializationDelegate = null;
protected StreamConfig configuration;
+ protected TypeSerializerWrapper<? extends Tuple, ? extends Tuple, OUT> typeWrapper;
protected StreamCollector<OUT> collector;
protected int instanceID;
protected String name;
@@ -68,19 +65,27 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
protected boolean isMutable;
protected Object function;
protected String functionName;
-
+
protected static int newComponent() {
numComponents++;
return numComponents;
}
+ @Override
+ public void registerInputOutput() {
+ initialize();
+ setInputsOutputs();
+ setInvokable();
+ setCollector();
+ }
+
protected void initialize() {
this.configuration = new StreamConfig(getTaskConfiguration());
this.name = configuration.getComponentName();
this.isMutable = configuration.getMutability();
this.functionName = configuration.getFunctionName();
this.function = configuration.getFunction();
-
+ this.typeWrapper = configuration.getTypeWrapper();
}
protected Collector<OUT> setCollector() {
@@ -96,40 +101,12 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
}
protected void setSerializers() {
- try {
- if (functionName.equals("flatMap")) {
- setSerializer(function, FlatMapFunction.class, 1);
- } else if (functionName.equals("map")) {
- setSerializer(function, MapFunction.class, 1);
- } else if (functionName.equals("batchReduce")) {
- setSerializer(function, GroupReduceFunction.class, 1);
- } else if (functionName.equals("filter")) {
- setSerializer(function, FilterFunction.class, 0);
- } else if (functionName.equals("source")) {
- setSerializer(function, UserSourceInvokable.class, 0);
- } else if (functionName.equals("coMap")) {
- setSerializer(function, CoMapFunction.class, 2);
- } else if (functionName.equals("elements")) {
- outTupleTypeInfo = new TupleTypeInfo<OUT>(TypeExtractor.getForObject(function));
-
- outTupleSerializer = new StreamRecordSerializer<OUT>(outTupleTypeInfo.createSerializer());
- outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(
- outTupleSerializer);
- } else {
- throw new Exception("Wrong operator name: " + functionName);
- }
- } catch (Exception e) {
- throw new StreamComponentException(e);
- }
-
+ setSerializer();
}
-
- @SuppressWarnings({ "rawtypes", "unchecked" })
- protected void setSerializer(Object function, Class<?> clazz, int typeParameter) {
- outTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
- typeParameter, null, null);
- outTupleSerializer = new StreamRecordSerializer(outTupleTypeInfo.createSerializer());
+ protected void setSerializer() {
+ outTupleTypeInfo = typeWrapper.getOutputTupleTypeInfo();
+ outTupleSerializer = new StreamRecordSerializer<OUT>(outTupleTypeInfo.createSerializer());
outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(outTupleSerializer);
}
@@ -137,7 +114,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
setSerializers();
setCollector();
-
+
int numberOfOutputs = configuration.getNumberOfOutputs();
for (int i = 0; i < numberOfOutputs; i++) {
@@ -148,7 +125,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
private void setPartitioner(int outputNumber,
List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
StreamPartitioner<OUT> outputPartitioner = null;
-
+
try {
outputPartitioner = configuration.getPartitioner(outputNumber);
@@ -189,8 +166,9 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
* Class of the invokable function
* @return The StreamComponent object
*/
- protected <T extends StreamComponentInvokable> T getInvokable() {
- return configuration.getUserInvokableObject();
+ @SuppressWarnings("unchecked")
+ protected <T extends StreamComponentInvokable<OUT>> T getInvokable() {
+ return (T) configuration.getUserInvokableObject();
}
protected <IN extends Tuple> MutableObjectIterator<StreamRecord<IN>> createInputIterator(
@@ -210,16 +188,8 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
return (T) SerializationUtils.deserialize(serializedObject);
}
-
- @Override
- public void registerInputOutput() {
- initialize();
- setInputsOutputs();
- setInvokable();
- }
-
protected abstract void setInputsOutputs();
-
+
protected abstract void setInvokable();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/BlockingQueueBroker.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/BlockingQueueBroker.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/BlockingQueueBroker.java
index c8eca70..52de3d1 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/BlockingQueueBroker.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/BlockingQueueBroker.java
@@ -25,13 +25,14 @@ import org.apache.flink.runtime.iterative.concurrent.Broker;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@SuppressWarnings("rawtypes")
-public class BlockingQueueBroker extends Broker<BlockingQueue<StreamRecord>>{
+public class BlockingQueueBroker extends Broker<BlockingQueue<StreamRecord>> {
/**
* Singleton instance
*/
private static final BlockingQueueBroker INSTANCE = new BlockingQueueBroker();
- private BlockingQueueBroker() {}
+ private BlockingQueueBroker() {
+ }
/**
* retrieve singleton instance
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
index 4d531f7..0c02c16 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -28,7 +28,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.common.functions.AbstractFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.MutableRecordReader;
@@ -45,8 +44,8 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
AbstractStreamComponent<OUT> {
private static final Log LOG = LogFactory.getLog(CoStreamTask.class);
- protected StreamRecordSerializer<IN1> inTupleSerializer1 = null;
- protected StreamRecordSerializer<IN2> inTupleSerializer2 = null;
+ protected StreamRecordSerializer<IN1> inTupleDeserializer1 = null;
+ protected StreamRecordSerializer<IN2> inTupleDeserializer2 = null;
private MutableReader<IOReadableWritable> inputs1;
private MutableReader<IOReadableWritable> inputs2;
@@ -54,24 +53,25 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
MutableObjectIterator<StreamRecord<IN2>> inputIter2;
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
- private CoInvokable<IN1, IN2, OUT> userFunction;
+ private CoInvokable<IN1, IN2, OUT> userInvokable;
private static int numTasks;
public CoStreamTask() {
outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
- userFunction = null;
+ userInvokable = null;
numTasks = newComponent();
instanceID = numTasks;
}
+ @Override
protected void setSerializers() {
String operatorName = configuration.getFunctionName();
Object function = configuration.getFunction();
try {
if (operatorName.equals("coMap")) {
- setSerializer(function, CoMapFunction.class, 2);
+ setSerializer();
setDeserializers(function, CoMapFunction.class);
} else {
throw new Exception("Wrong operator name!");
@@ -83,13 +83,11 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
@SuppressWarnings({ "unchecked", "rawtypes" })
private void setDeserializers(Object function, Class<? extends AbstractFunction> clazz) {
- TupleTypeInfo<IN1> inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz,
- function.getClass(), 0, null, null);
- inTupleSerializer1 = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
+ TupleTypeInfo<IN1> inTupleTypeInfo = (TupleTypeInfo<IN1>) typeWrapper.getInputTupleTypeInfo1();
+ inTupleDeserializer1 = new StreamRecordSerializer<IN1>(inTupleTypeInfo.createSerializer());
- inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
- 1, null, null);
- inTupleSerializer2 = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
+ inTupleTypeInfo = (TupleTypeInfo<IN1>) typeWrapper.getInputTupleTypeInfo2();
+ inTupleDeserializer2 = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
}
@Override
@@ -97,15 +95,15 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
setConfigOutputs(outputs);
setConfigInputs();
- inputIter1 = createInputIterator(inputs1, inTupleSerializer1);
- inputIter2 = createInputIterator(inputs2, inTupleSerializer2);
+ inputIter1 = createInputIterator(inputs1, inTupleDeserializer1);
+ inputIter2 = createInputIterator(inputs2, inTupleDeserializer2);
}
-
+
@Override
protected void setInvokable() {
- userFunction = getInvokable();
- userFunction.initialize(collector, inputIter1, inTupleSerializer1, inputIter2,
- inTupleSerializer2, isMutable);
+ userInvokable = getInvokable();
+ userInvokable.initialize(collector, inputIter1, inTupleDeserializer1, inputIter2,
+ inTupleDeserializer2, isMutable);
}
protected void setConfigInputs() throws StreamComponentException {
@@ -156,7 +154,7 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
output.initializeSerializers();
}
- userFunction.invoke();
+ userInvokable.invoke();
if (LOG.isDebugEnabled()) {
LOG.debug("TASK " + name + " invoke finished with instance id " + instanceID);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/799424d1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
index 0b5b377..8355b78 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
@@ -19,62 +19,38 @@
package org.apache.flink.streaming.api.streamcomponent;
-import org.apache.flink.api.common.functions.AbstractFunction;
-import org.apache.flink.api.java.functions.FilterFunction;
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.functions.GroupReduceFunction;
-import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.MutableRecordReader;
import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.util.MutableObjectIterator;
-public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT extends Tuple> extends
- AbstractStreamComponent<OUT> {
+public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT extends Tuple>
+ extends AbstractStreamComponent<OUT> {
protected StreamRecordSerializer<IN> inTupleSerializer = null;
protected MutableObjectIterator<StreamRecord<IN>> inputIter;
protected MutableReader<IOReadableWritable> inputs;
protected void setDeserializers() {
- try {
- if (functionName.equals("flatMap")) {
- setDeserializer(function, FlatMapFunction.class);
- } else if (functionName.equals("map")) {
- setDeserializer(function, MapFunction.class);
- } else if (functionName.equals("batchReduce")) {
- setDeserializer(function, GroupReduceFunction.class);
- } else if (functionName.equals("filter")) {
- setDeserializer(function, FilterFunction.class);
- } else if (functionName.equals("source")) {
- setSerializer(function, UserSourceInvokable.class, 0);
- } else if (functionName.equals("sink")) {
- setDeserializer(function, SinkFunction.class);
- } else {
- throw new Exception("Wrong operator name: " + functionName);
- }
-
- } catch (Exception e) {
- throw new StreamComponentException(e);
+ if (functionName.equals(SOURCE)) {
+ setSerializer();
+ } else {
+ setDeserializer();
}
}
- @SuppressWarnings({ "unchecked", "rawtypes" })
- private void setDeserializer(Object function, Class<? extends AbstractFunction> clazz) {
- TupleTypeInfo<IN> inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
- 0, null, null);
-
- inTupleSerializer = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
+ @SuppressWarnings("unchecked")
+ private void setDeserializer() {
+ TupleTypeInfo<IN> inTupleTypeInfo = (TupleTypeInfo<IN>) typeWrapper
+ .getInputTupleTypeInfo1();
+ inTupleSerializer = new StreamRecordSerializer<IN>(inTupleTypeInfo.createSerializer());
}
-
+
@SuppressWarnings("unchecked")
protected void setSinkSerializer() {
if (outSerializationDelegate != null) {
@@ -87,7 +63,7 @@ public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT e
@SuppressWarnings("unchecked")
protected void setConfigInputs() throws StreamComponentException {
setDeserializers();
-
+
int numberOfInputs = configuration.getNumberOfInputs();
if (numberOfInputs < 2) {
[25/51] [abbrv] git commit: [streaming] JSONParseFlatMap added to
examples
Posted by se...@apache.org.
[streaming] JSONParseFlatMap added to examples
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/ee7c4a83
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/ee7c4a83
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/ee7c4a83
Branch: refs/heads/master
Commit: ee7c4a8310ec7e3649a803edf42a532c580ea0b1
Parents: 799424d
Author: Eszes Dávid <es...@gmail.com>
Authored: Fri Aug 1 11:53:12 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:11 2014 +0200
----------------------------------------------------------------------
.../flink-streaming-connectors/pom.xml | 6 ++
.../flink-streaming-core/pom.xml | 8 ++
.../examples/function/JSONParseFlatMap.java | 99 ++++++++++++++++++++
3 files changed, 113 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ee7c4a83/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
index 8db610f..23e3fef 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
@@ -43,6 +43,12 @@ under the License.
</dependency>
<dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-examples</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
+ <dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka_2.10</artifactId>
<version>0.8.0</version>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ee7c4a83/flink-addons/flink-streaming/flink-streaming-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/pom.xml b/flink-addons/flink-streaming/flink-streaming-core/pom.xml
index 04b9372..73b8158 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-core/pom.xml
@@ -35,11 +35,19 @@ under the License.
<packaging>jar</packaging>
<dependencies>
+
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
<version>3.1</version>
</dependency>
+
+ <dependency>
+ <groupId>org.apache.sling</groupId>
+ <artifactId>org.apache.sling.commons.json</artifactId>
+ <version>2.0.6</version>
+ </dependency>
+
</dependencies>
<build>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ee7c4a83/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
new file mode 100644
index 0000000..267d035
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
@@ -0,0 +1,99 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.examples.function;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.flink.api.java.functions.FlatMapFunction;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.sling.commons.json.JSONException;
+import org.apache.sling.commons.json.JSONObject;
+
+/**
+ * Abstract class derived from {@link FlatMapFunction} to handle JSON files.
+ * @param <IN>
+ * Type of the input elements.
+ * @param <OUT>
+ * Type of the returned elements.
+ */
+public abstract class JSONParseFlatMap<IN, OUT> extends
+ FlatMapFunction<IN, OUT> {
+
+ private static final long serialVersionUID = 1L;
+ private static final Log LOG = LogFactory.getLog(DataStream.class);
+
+ /**
+ * Get the value of a field in a JSON text.
+ * @param jsonText
+ * The JSON text in which the field is searched.
+ * @param field
+ * The field which is searched in the JSON text.
+ * In case of embedded records fields have to be referred separated by dots.
+ * @return
+ * The value of the given field if it exists. Otherwise function returns with null.
+ */
+ public String getField(String jsonText, String field) {
+ JSONObject jo = null;
+ try {
+ jo = new JSONObject(jsonText);
+ } catch (JSONException e) {
+ if (LOG.isErrorEnabled()) {
+ LOG.error("Input string is not proper", e);
+ }
+ return null;
+ }
+
+ try {
+ String[] fieldArray = field.split("[.]");
+ int length = fieldArray.length;
+
+ return findInnerField(jo, fieldArray, length).getString(
+ fieldArray[length - 1]);
+
+ } catch (JSONException e) {
+ if (LOG.isErrorEnabled()) {
+ LOG.error("Field " + field + " not found");
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Find an embedded JSON code associated with the given key (fieldArray).
+ * @param jo
+ * JSONObject in which we search.
+ * @param fieldArray
+ * String array identifying the field.
+ * @param length
+ * Length of the array.
+ * @return
+ * the searched embedded JSONObject if it exists.
+ * @throws JSONException
+ * if the key is not found.
+ */
+ private JSONObject findInnerField(JSONObject jo, String[] fieldArray,
+ int length) throws JSONException {
+
+ for (int i = 0; i <= length - 2; i++) {
+ jo = jo.getJSONObject(fieldArray[i]);
+ }
+ return jo;
+ }
+}
[31/51] [abbrv] git commit: [streaming] Added new DataStream types
for operators
Posted by se...@apache.org.
[streaming] Added new DataStream types for operators
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/0465d300
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/0465d300
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/0465d300
Branch: refs/heads/master
Commit: 0465d300cbc6cf6c69dc4da2b64cecefaa2d77db
Parents: f932700
Author: gyfora <gy...@gmail.com>
Authored: Mon Aug 4 11:36:00 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:50 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 73 ++++++++++----------
.../streaming/api/IterativeDataStream.java | 4 +-
.../api/SingleInputStreamOperator.java | 46 ++++++++++++
.../flink/streaming/api/StreamOperator.java | 11 ++-
.../streaming/api/TwoInputStreamOperator.java | 49 +++++++++++++
.../testdata_checksum/ASTopology.data.md5 | 1 -
.../testdata_checksum/MovieLens100k.data.md5 | 1 -
.../resources/testdata_checksum/hamlet.txt.md5 | 1 -
.../testdata_checksum/terainput.txt.md5 | 1 -
9 files changed, 142 insertions(+), 45 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0465d300/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index 10f8114..430f09b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -174,9 +174,9 @@ public class DataStream<T> {
throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
}
this.degreeOfParallelism = dop;
-
+
jobGraphBuilder.setParallelism(id, degreeOfParallelism);
-
+
return this;
}
@@ -319,7 +319,7 @@ public class DataStream<T> {
* output type
* @return The transformed DataStream.
*/
- public <R> StreamOperator<T, R> map(RichMapFunction<T, R> mapper) {
+ public <R> StreamOperator<R> map(RichMapFunction<T, R> mapper) {
return addFunction("map", mapper, new FunctionTypeWrapper<T, Tuple, R>(mapper,
RichMapFunction.class, 0, -1, 1), new MapInvokable<T, R>(mapper));
}
@@ -338,7 +338,7 @@ public class DataStream<T> {
* output type
* @return The transformed DataStream.
*/
- public <R> StreamOperator<T, R> flatMap(RichFlatMapFunction<T, R> flatMapper) {
+ public <R> StreamOperator<R> flatMap(RichFlatMapFunction<T, R> flatMapper) {
return addFunction("flatMap", flatMapper, new FunctionTypeWrapper<T, Tuple, R>(flatMapper,
RichFlatMapFunction.class, 0, -1, 1), new FlatMapInvokable<T, R>(flatMapper));
}
@@ -358,7 +358,7 @@ public class DataStream<T> {
* {@link CoMapFunction#map2(Tuple)}
* @return The transformed DataStream
*/
- public <T2, R> DataStream<R> coMapWith(CoMapFunction<T, T2, R> coMapper,
+ public <T2, R> StreamOperator<R> coMapWith(CoMapFunction<T, T2, R> coMapper,
DataStream<T2> otherStream) {
return addCoFunction("coMap", this.copy(), otherStream.copy(), coMapper,
new FunctionTypeWrapper<T, T2, R>(coMapper, CoMapFunction.class, 0, 1, 2),
@@ -381,7 +381,7 @@ public class DataStream<T> {
* output type
* @return The modified DataStream.
*/
- public <R> StreamOperator<T, R> batchReduce(RichGroupReduceFunction<T, R> reducer, int batchSize) {
+ public <R> StreamOperator<R> batchReduce(RichGroupReduceFunction<T, R> reducer, int batchSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
RichGroupReduceFunction.class, 0, -1, 1), new BatchReduceInvokable<T, R>(reducer,
batchSize));
@@ -404,8 +404,7 @@ public class DataStream<T> {
* output type
* @return The modified DataStream.
*/
- public <R> StreamOperator<T, R> windowReduce(RichGroupReduceFunction<T, R> reducer,
- long windowSize) {
+ public <R> StreamOperator<R> windowReduce(RichGroupReduceFunction<T, R> reducer, long windowSize) {
return addFunction("batchReduce", reducer, new FunctionTypeWrapper<T, Tuple, R>(reducer,
RichGroupReduceFunction.class, 0, -1, 1), new WindowReduceInvokable<T, R>(reducer,
windowSize));
@@ -422,7 +421,7 @@ public class DataStream<T> {
* DataSet.
* @return The filtered DataStream.
*/
- public StreamOperator<T, T> filter(RichFilterFunction<T> filter) {
+ public StreamOperator<T> filter(RichFilterFunction<T> filter) {
return addFunction("filter", filter, new FunctionTypeWrapper<T, Tuple, T>(filter,
RichFilterFunction.class, 0, -1, 0), new FilterInvokable<T>(filter));
}
@@ -766,10 +765,10 @@ public class DataStream<T> {
protected <R> DataStream<T> addIterationSource(String iterationID) {
DataStream<R> returnStream = new DataStream<R>(environment, "iterationSource");
-
+
jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
degreeOfParallelism);
-
+
return this.copy();
}
@@ -783,70 +782,70 @@ public class DataStream<T> {
* the user defined function
* @param functionInvokable
* the wrapping JobVertex instance
- * @param <T>
- * type of the input stream
* @param <R>
* type of the return stream
* @return the data stream constructed
*/
- private <R> StreamOperator<T, R> addFunction(String functionName,
+ private <R> StreamOperator<R> addFunction(String functionName,
final AbstractRichFunction function, TypeSerializerWrapper<T, Tuple, R> typeWrapper,
UserTaskInvokable<T, R> functionInvokable) {
-
+
DataStream<T> inputStream = this.copy();
- StreamOperator<T, R> returnStream = new StreamOperator<T, R>(environment, functionName);
-
+ StreamOperator<R> returnStream = new SingleInputStreamOperator<T, R>(environment,
+ functionName);
+
try {
jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, typeWrapper,
functionName, SerializationUtils.serialize(function), degreeOfParallelism);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize user defined function");
}
-
+
connectGraph(inputStream, returnStream.getId(), 0);
-
+
if (inputStream instanceof IterativeDataStream) {
returnStream.addIterationSource(((IterativeDataStream<T>) inputStream).iterationID
.toString());
}
-
+
if (inputStream instanceof NamedDataStream) {
returnStream.name(inputStream.userDefinedName);
}
-
+
return returnStream;
}
- protected <T1, T2, R> DataStream<R> addCoFunction(String functionName,
+ protected <T1, T2, R> StreamOperator<R> addCoFunction(String functionName,
DataStream<T1> inputStream1, DataStream<T2> inputStream2,
final AbstractRichFunction function, TypeSerializerWrapper<T1, T2, R> typeWrapper,
CoInvokable<T1, T2, R> functionInvokable) {
-
- DataStream<R> returnStream = new DataStream<R>(environment, functionName);
-
+
+ StreamOperator<R> returnStream = new TwoInputStreamOperator<T1, T2, R>(environment,
+ functionName);
+
try {
jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, typeWrapper,
functionName, SerializationUtils.serialize(function), degreeOfParallelism);
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize user defined function");
}
-
+
connectGraph(inputStream1, returnStream.getId(), 1);
connectGraph(inputStream2, returnStream.getId(), 2);
-
+
if ((inputStream1 instanceof NamedDataStream) && (inputStream2 instanceof NamedDataStream)) {
throw new RuntimeException("An operator cannot have two names");
} else {
if (inputStream1 instanceof NamedDataStream) {
returnStream.name(inputStream1.userDefinedName);
}
-
+
if (inputStream2 instanceof NamedDataStream) {
returnStream.name(inputStream2.userDefinedName);
}
}
// TODO consider iteration
-
+
return returnStream;
}
@@ -864,10 +863,10 @@ public class DataStream<T> {
if (name == "") {
throw new IllegalArgumentException("User defined name must not be empty string");
}
-
+
userDefinedName = name;
jobGraphBuilder.setUserDefinedName(id, name);
-
+
return this;
}
@@ -890,11 +889,11 @@ public class DataStream<T> {
private DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
DataStream<T> returnStream = this.copy();
-
+
for (DataStream<T> stream : returnStream.connectedStreams) {
stream.partitioner = partitioner;
}
-
+
return returnStream;
}
@@ -938,7 +937,7 @@ public class DataStream<T> {
private DataStream<T> addSink(DataStream<T> inputStream, SinkFunction<T> sinkFunction,
TypeSerializerWrapper<T, Tuple, T> typeWrapper) {
DataStream<T> returnStream = new DataStream<T>(environment, "sink");
-
+
try {
jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<T>(sinkFunction),
typeWrapper, "sink", SerializationUtils.serialize(sinkFunction),
@@ -946,13 +945,13 @@ public class DataStream<T> {
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize SinkFunction");
}
-
+
inputStream.connectGraph(inputStream, returnStream.getId(), 0);
-
+
if (this.copy() instanceof NamedDataStream) {
returnStream.name(inputStream.userDefinedName);
}
-
+
return returnStream;
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0465d300/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
index d5a5f78..71e129c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/IterativeDataStream.java
@@ -23,12 +23,12 @@ import org.apache.flink.streaming.partitioner.ForwardPartitioner;
/**
* The iterative data stream represents the start of an iteration in a
- * DataStream.
+ * {@link DataStream}.
*
* @param <T>
* Type of the DataStream
*/
-public class IterativeDataStream<T> extends StreamOperator<T, T> {
+public class IterativeDataStream<T> extends SingleInputStreamOperator<T, T> {
static Integer iterationCount = 0;
protected Integer iterationID;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0465d300/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SingleInputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SingleInputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SingleInputStreamOperator.java
new file mode 100755
index 0000000..43dfe36
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/SingleInputStreamOperator.java
@@ -0,0 +1,46 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api;
+
+/**
+ * The SingleInputStreamOperator represents a {@link DataStream} transformed
+ * with some user defined operator.
+ *
+ * @param <IN>
+ * Input type of the operator.
+ * @param <OUT>
+ * Output Type of the operator.
+ */
+public class SingleInputStreamOperator<IN, OUT> extends StreamOperator<OUT> {
+
+ protected SingleInputStreamOperator(StreamExecutionEnvironment environment, String operatorType) {
+ super(environment, operatorType);
+ }
+
+ protected SingleInputStreamOperator(DataStream<OUT> dataStream) {
+ super(dataStream);
+ }
+
+ @Override
+ protected DataStream<OUT> copy() {
+ return new SingleInputStreamOperator<IN, OUT>(this);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0465d300/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
index 5ecd930..a39823c 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamOperator.java
@@ -19,7 +19,14 @@
package org.apache.flink.streaming.api;
-public class StreamOperator<IN, OUT> extends DataStream<OUT> {
+/**
+ * The StreamOperator represents a {@link DataStream} transformed with some user
+ * defined operator.
+ *
+ * @param <OUT>
+ * Output Type of the operator.
+ */
+public class StreamOperator<OUT> extends DataStream<OUT> {
protected StreamOperator(StreamExecutionEnvironment environment, String operatorType) {
super(environment, operatorType);
@@ -31,7 +38,7 @@ public class StreamOperator<IN, OUT> extends DataStream<OUT> {
@Override
protected DataStream<OUT> copy() {
- return new StreamOperator<IN, OUT>(this);
+ return new StreamOperator<OUT>(this);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0465d300/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TwoInputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TwoInputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TwoInputStreamOperator.java
new file mode 100755
index 0000000..a459dbf
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TwoInputStreamOperator.java
@@ -0,0 +1,49 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api;
+
+/**
+ * The TwoInputStreamOperator represents a {@link StreamOperator} with two
+ * inputs.
+ *
+ * @param <IN1>
+ * Type of the first input.
+ *
+ * @param <IN2>
+ * Type of the second input.
+ * @param <OUT>
+ * Output Type of the operator.
+ */
+public class TwoInputStreamOperator<IN1, IN2, OUT> extends StreamOperator<OUT> {
+
+ protected TwoInputStreamOperator(StreamExecutionEnvironment environment, String operatorType) {
+ super(environment, operatorType);
+ }
+
+ protected TwoInputStreamOperator(DataStream<OUT> dataStream) {
+ super(dataStream);
+ }
+
+ @Override
+ protected DataStream<OUT> copy() {
+ return new TwoInputStreamOperator<IN1, IN2, OUT>(this);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0465d300/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
deleted file mode 100644
index 2c386b7..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
+++ /dev/null
@@ -1 +0,0 @@
-f1b947a26b33b32f1de2cdd841f7b4c8
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0465d300/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
deleted file mode 100644
index 6499b43..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
+++ /dev/null
@@ -1 +0,0 @@
-6e47046882bad158b0efbb84cd5cb987
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0465d300/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
deleted file mode 100644
index 6526a51..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
+++ /dev/null
@@ -1 +0,0 @@
-4bb8c10cdde12a4953250423266465cc
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0465d300/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
deleted file mode 100644
index 365f210..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
+++ /dev/null
@@ -1 +0,0 @@
-7002e15fe547614160a0df6f22a5b8d0
[46/51] [abbrv] git commit: [streaming] Exception handling update &
minor refactor
Posted by se...@apache.org.
[streaming] Exception handling update & minor refactor
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/35cf874c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/35cf874c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/35cf874c
Branch: refs/heads/master
Commit: 35cf874c36fc522e9fae6af4625901ee3c3bc9a3
Parents: e6766fd
Author: ghermann <re...@gmail.com>
Authored: Mon Aug 11 19:56:35 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:43 2014 +0200
----------------------------------------------------------------------
.../streaming/connectors/rabbitmq/RMQSink.java | 2 +-
.../flink/streaming/api/StreamConfig.java | 10 ---
.../api/collector/DirectedStreamCollector.java | 6 +-
.../environment/RemoteStreamEnvironment.java | 6 +-
.../streaming/api/invokable/SinkInvokable.java | 9 ++-
.../api/invokable/SourceInvokable.java | 1 +
.../api/invokable/StreamComponentInvokable.java | 3 +-
.../api/invokable/StreamRecordInvokable.java | 20 ++++-
.../operator/BatchGroupReduceInvokable.java | 22 ++++--
.../operator/BatchReduceInvokable.java | 1 +
.../api/invokable/operator/FilterInvokable.java | 12 ++-
.../invokable/operator/FlatMapInvokable.java | 9 ++-
.../operator/GroupReduceInvokable.java | 15 +++-
.../api/invokable/operator/MapInvokable.java | 9 ++-
.../operator/StreamReduceInvokable.java | 9 ++-
.../operator/WindowGroupReduceInvokable.java | 17 +++-
.../api/invokable/operator/co/CoInvokable.java | 2 -
.../AbstractStreamComponent.java | 82 ++++++++++++++++----
.../api/streamcomponent/CoStreamTask.java | 33 +-------
.../streamcomponent/StreamIterationSource.java | 20 ++---
.../api/streamcomponent/StreamRecordWriter.java | 23 +++---
.../api/streamcomponent/StreamSource.java | 37 +--------
.../api/streamcomponent/StreamTask.java | 38 ++-------
.../flink/streaming/state/StateManager.java | 15 ++--
.../flink/streaming/util/ClusterUtil.java | 14 ++--
.../flink/streaming/util/TestDataUtil.java | 20 ++---
26 files changed, 229 insertions(+), 206 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index 12b6341..4a89243 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -60,7 +60,7 @@ public abstract class RMQSink<IN> implements SinkFunction<IN> {
channel = connection.createChannel();
} catch (IOException e) {
- e.printStackTrace();
+ throw new RuntimeException(e);
}
initDone = true;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index 3d124c2..4dd4cc1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -119,12 +119,6 @@ public class StreamConfig {
}
}
- // @SuppressWarnings("unchecked")
- // public <T extends StreamComponentInvokable> Class<? extends T>
- // getUserInvokableClass() {
- // return (Class<? extends T>) config.getClass(USER_FUNCTION, null);
- // }
-
public <T> StreamComponentInvokable<T> getUserInvokableObject() {
try {
return deserializeObject(config.getBytes(SERIALIZEDUDF, null));
@@ -156,10 +150,6 @@ public class StreamConfig {
}
}
- // public void setFunctionName(String functionName) {
- // config.setString(FUNCTION_NAME, functionName);
- // }
-
public String getFunctionName() {
return config.getString(FUNCTION_NAME, "");
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
index 285a7b4..9750d44 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
@@ -40,7 +40,7 @@ import org.apache.flink.util.StringUtils;
public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
OutputSelector<OUT> outputSelector;
- private static final Log log = LogFactory.getLog(DirectedStreamCollector.class);
+ private static final Log LOG = LogFactory.getLog(DirectedStreamCollector.class);
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> emitted;
/**
@@ -97,8 +97,8 @@ public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
}
}
} catch (Exception e) {
- if (log.isErrorEnabled()) {
- log.error(String.format("Emit to %s failed due to: %s", outputName,
+ if (LOG.isErrorEnabled()) {
+ LOG.error(String.format("Emit to %s failed due to: %s", outputName,
StringUtils.stringifyException(e)));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index 19a2d48..ec6bc6f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -34,7 +34,7 @@ import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.jobgraph.JobGraph;
public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
- private static final Log log = LogFactory.getLog(RemoteStreamEnvironment.class);
+ private static final Log LOG = LogFactory.getLog(RemoteStreamEnvironment.class);
private String host;
private int port;
@@ -72,8 +72,8 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
@Override
public void execute() {
- if (log.isInfoEnabled()) {
- log.info("Running remotely at " + host + ":" + port);
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Running remotely at " + host + ":" + port);
}
JobGraph jobGraph = jobGraphBuilder.getJobGraph();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index 887df8b..9219f80 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -34,7 +34,7 @@ public class SinkInvokable<IN> extends StreamRecordInvokable<IN, IN> {
@Override
protected void immutableInvoke() throws Exception {
while ((reuse = recordIterator.next(reuse)) != null) {
- sinkFunction.invoke((IN) reuse.getObject());
+ callUserFunctionAndLogException();
resetReuse();
}
}
@@ -42,8 +42,13 @@ public class SinkInvokable<IN> extends StreamRecordInvokable<IN, IN> {
@Override
protected void mutableInvoke() throws Exception {
while ((reuse = recordIterator.next(reuse)) != null) {
- sinkFunction.invoke((IN) reuse.getObject());
+ callUserFunctionAndLogException();
}
}
+ @Override
+ protected void callUserFunction() throws Exception {
+ sinkFunction.invoke((IN) reuse.getObject());
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
index a4be1e8..666427f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
@@ -35,6 +35,7 @@ public class SourceInvokable<OUT> extends StreamComponentInvokable<OUT> implemen
this.sourceFunction = sourceFunction;
}
+ @Override
public void invoke() throws Exception {
sourceFunction.invoke(collector);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
index ed718f1..4b49252 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamComponentInvokable.java
@@ -65,5 +65,6 @@ public abstract class StreamComponentInvokable<OUT> extends AbstractRichFunction
((RichFunction) userFunction).close();
}
}
-
+
+ public abstract void invoke() throws Exception;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
index 27dc05a..d4fc92c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamRecordInvokable.java
@@ -21,11 +21,14 @@ package org.apache.flink.streaming.api.invokable;
import java.io.IOException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
+import org.apache.flink.util.StringUtils;
public abstract class StreamRecordInvokable<IN, OUT> extends
StreamComponentInvokable<OUT> {
@@ -35,6 +38,7 @@ public abstract class StreamRecordInvokable<IN, OUT> extends
}
private static final long serialVersionUID = 1L;
+ private static final Log LOG = LogFactory.getLog(StreamComponentInvokable.class);
protected MutableObjectIterator<StreamRecord<IN>> recordIterator;
StreamRecordSerializer<IN> serializer;
@@ -59,7 +63,7 @@ public abstract class StreamRecordInvokable<IN, OUT> extends
try {
reuse = recordIterator.next(reuse);
} catch (IOException e) {
- e.printStackTrace();
+ throw new RuntimeException(e);
}
return reuse;
}
@@ -68,6 +72,20 @@ public abstract class StreamRecordInvokable<IN, OUT> extends
protected abstract void mutableInvoke() throws Exception;
+ protected abstract void callUserFunction() throws Exception;
+
+ protected void callUserFunctionAndLogException() {
+ try {
+ callUserFunction();
+ } catch (Exception e) {
+ if (LOG.isErrorEnabled()) {
+ LOG.error(String.format("Calling user function failed due to: %s",
+ StringUtils.stringifyException(e)));
+ }
+ }
+ }
+
+ @Override
public void invoke() throws Exception {
if (this.isMutable) {
mutableInvoke();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
index c64bd33..327ddaf 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
@@ -27,6 +27,8 @@ import org.apache.flink.streaming.state.MutableTableState;
public class BatchGroupReduceInvokable<IN> extends BatchReduceInvokable<IN, IN> {
+ private static final long serialVersionUID = 1L;
+
int keyPosition;
protected ReduceFunction<IN> reducer;
private Iterator<StreamRecord<IN>> iterator;
@@ -40,18 +42,22 @@ public class BatchGroupReduceInvokable<IN> extends BatchReduceInvokable<IN, IN>
values = new MutableTableState<Object, IN>();
}
+ private IN reduced;
+ private IN nextValue;
+ private IN currentValue;
+
@Override
- protected void reduce() throws Exception {
+ protected void reduce() {
iterator = state.getStreamRecordIterator();
while (iterator.hasNext()) {
StreamRecord<IN> nextRecord = iterator.next();
- IN nextValue = nextRecord.getObject();
+ nextValue = nextRecord.getObject();
Object key = nextRecord.getField(keyPosition);
- IN currentValue = values.get(key);
+ currentValue = values.get(key);
if (currentValue != null) {
- IN reduced = reducer.reduce(currentValue, nextValue);
+ callUserFunctionAndLogException();
values.put(key, reduced);
collector.collect(reduced);
} else {
@@ -59,12 +65,12 @@ public class BatchGroupReduceInvokable<IN> extends BatchReduceInvokable<IN, IN>
collector.collect(nextValue);
}
}
- System.out.println(values);
values.clear();
- System.out.println(values);
-
}
- private static final long serialVersionUID = 1L;
+ @Override
+ protected void callUserFunction() throws Exception {
+ reduced = reducer.reduce(currentValue, nextValue);
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 3d1d813..a6516f8 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -71,4 +71,5 @@ public class BatchReduceInvokable<IN, OUT> extends StreamReduceInvokable<IN, OUT
}
}
+
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
index 388920c..9e0edd7 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
@@ -33,10 +33,13 @@ public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
this.filterFunction = filterFunction;
}
+ private boolean canCollect;
+
@Override
protected void immutableInvoke() throws Exception {
while ((reuse = recordIterator.next(reuse)) != null) {
- if (filterFunction.filter(reuse.getObject())) {
+ callUserFunctionAndLogException();
+ if (canCollect) {
collector.collect(reuse.getObject());
}
resetReuse();
@@ -46,10 +49,15 @@ public class FilterInvokable<IN> extends UserTaskInvokable<IN, IN> {
@Override
protected void mutableInvoke() throws Exception {
while ((reuse = recordIterator.next(reuse)) != null) {
- if (filterFunction.filter(reuse.getObject())) {
+ callUserFunctionAndLogException();
+ if (canCollect) {
collector.collect(reuse.getObject());
}
}
}
+ @Override
+ protected void callUserFunction() throws Exception {
+ canCollect = filterFunction.filter(reuse.getObject());
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
index 4cb4712..0b4b4d1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
@@ -35,7 +35,7 @@ public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
@Override
protected void immutableInvoke() throws Exception {
while ((reuse = recordIterator.next(reuse)) != null) {
- flatMapper.flatMap(reuse.getObject(), collector);
+ callUserFunctionAndLogException();
resetReuse();
}
}
@@ -43,8 +43,13 @@ public class FlatMapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
@Override
protected void mutableInvoke() throws Exception {
while ((reuse = recordIterator.next(reuse)) != null) {
- flatMapper.flatMap(reuse.getObject(), collector);
+ callUserFunctionAndLogException();
}
}
+ @Override
+ protected void callUserFunction() throws Exception {
+ flatMapper.flatMap(reuse.getObject(), collector);
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
index 277078b..43ab483 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
@@ -52,12 +52,16 @@ public class GroupReduceInvokable<IN> extends UserTaskInvokable<IN, IN> {
}
}
+ private IN reduced;
+ private IN nextValue;
+ private IN currentValue;
+
private void reduce() throws Exception {
Object key = reuse.getField(keyPosition);
- IN currentValue = values.get(key);
- IN nextValue = reuse.getObject();
+ currentValue = values.get(key);
+ nextValue = reuse.getObject();
if (currentValue != null) {
- IN reduced = reducer.reduce(currentValue, nextValue);
+ callUserFunctionAndLogException();
values.put(key, reduced);
collector.collect(reduced);
} else {
@@ -66,4 +70,9 @@ public class GroupReduceInvokable<IN> extends UserTaskInvokable<IN, IN> {
}
}
+ @Override
+ protected void callUserFunction() throws Exception {
+ reduced = reducer.reduce(currentValue, nextValue);
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
index 53e85e0..2e501ca 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
@@ -35,7 +35,7 @@ public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
@Override
protected void immutableInvoke() throws Exception {
while ((reuse = recordIterator.next(reuse)) != null) {
- collector.collect(mapper.map(reuse.getObject()));
+ callUserFunctionAndLogException();
resetReuse();
}
}
@@ -43,7 +43,12 @@ public class MapInvokable<IN, OUT> extends UserTaskInvokable<IN, OUT> {
@Override
protected void mutableInvoke() throws Exception {
while ((reuse = recordIterator.next(reuse)) != null) {
- collector.collect(mapper.map(reuse.getObject()));
+ callUserFunctionAndLogException();
}
}
+
+ @Override
+ protected void callUserFunction() throws Exception {
+ collector.collect(mapper.map(reuse.getObject()));
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
index f6515dc..c73bb95 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
@@ -71,11 +71,16 @@ public abstract class StreamReduceInvokable<IN, OUT> extends UserTaskInvokable<I
}
}
- protected void reduce() throws Exception {
+ protected void reduce() {
userIterator = state.getIterator();
+ callUserFunctionAndLogException();
+ }
+
+ @Override
+ protected void callUserFunction() throws Exception {
reducer.reduce(userIterable, collector);
}
-
+
private void collectOneUnit() {
ArrayList<StreamRecord<IN>> list;
list = new ArrayList<StreamRecord<IN>>(listSize);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
index 065df25..ac57220 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
@@ -40,18 +40,22 @@ public class WindowGroupReduceInvokable<IN> extends WindowReduceInvokable<IN, IN
values = new MutableTableState<Object, IN>();
}
+ private IN reduced;
+ private IN nextValue;
+ private IN currentValue;
+
@Override
- protected void reduce() throws Exception {
+ protected void reduce() {
iterator = state.getStreamRecordIterator();
while (iterator.hasNext()) {
StreamRecord<IN> nextRecord = iterator.next();
- IN nextValue = nextRecord.getObject();
+ nextValue = nextRecord.getObject();
Object key = nextRecord.getField(keyPosition);
- IN currentValue = values.get(key);
+ currentValue = values.get(key);
if (currentValue != null) {
- IN reduced = reducer.reduce(currentValue, nextValue);
+ callUserFunctionAndLogException();
values.put(key, reduced);
collector.collect(reduced);
} else {
@@ -62,6 +66,11 @@ public class WindowGroupReduceInvokable<IN> extends WindowReduceInvokable<IN, IN
values.clear();
}
+ @Override
+ protected void callUserFunction() throws Exception {
+ reduced = reducer.reduce(currentValue, nextValue);
+ }
+
private static final long serialVersionUID = 1L;
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
index 1e8623c..8e002ee 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
@@ -65,6 +65,4 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends
this.reuse1 = serializer1.createInstance();
this.reuse2 = serializer2.createInstance();
}
-
- public abstract void invoke() throws Exception;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index c973894..9171a00 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -20,6 +20,7 @@
package org.apache.flink.streaming.api.streamcomponent;
import java.io.IOException;
+import java.util.LinkedList;
import java.util.List;
import org.apache.commons.lang.SerializationUtils;
@@ -54,7 +55,8 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
protected TypeInformation<OUT> outTypeInfo = null;
protected StreamRecordSerializer<OUT> outSerializer = null;
protected SerializationDelegate<StreamRecord<OUT>> outSerializationDelegate = null;
-
+ protected OutputHandler outputHandler = createEmptyOutputHandler();
+
protected StreamConfig configuration;
protected TypeSerializerWrapper<?, ?, OUT> typeWrapper;
protected StreamCollector<OUT> collector;
@@ -79,6 +81,71 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
setCollector();
}
+ protected class OutputHandler {
+ private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
+
+ public OutputHandler() {
+ this.outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+ }
+
+ public List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> getOutputs() {
+ return outputs;
+ }
+
+ public void setConfigOutputs() {
+ setSerializers();
+ setCollector();
+
+ int numberOfOutputs = configuration.getNumberOfOutputs();
+ bufferTimeout = configuration.getBufferTimeout();
+
+ for (int i = 0; i < numberOfOutputs; i++) {
+ setPartitioner(i, outputs);
+ }
+ }
+
+ public void flushOutputs() throws IOException, InterruptedException {
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
+ output.flush();
+ }
+ }
+
+ public void initializeOutputSerializers() {
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputHandler.getOutputs()) {
+ output.initializeSerializers();
+ }
+ }
+
+ public void invokeUserFunction(String componentTypeName, StreamComponentInvokable<OUT> userInvokable) throws IOException, InterruptedException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(componentTypeName + " " + name + " invoked with instance id "
+ + instanceID);
+ }
+
+ initializeOutputSerializers();
+
+ try {
+ userInvokable.open(getTaskConfiguration());
+ userInvokable.invoke();
+ userInvokable.close();
+ } catch (Exception e) {
+ flushOutputs();
+ throw new RuntimeException(e);
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(componentTypeName + " " + name + " invoke finished with instance id "
+ + instanceID);
+ }
+
+ flushOutputs();
+ }
+ }
+
+ private OutputHandler createEmptyOutputHandler() {
+ return new OutputHandler();
+ }
+
protected void initialize() {
this.configuration = new StreamConfig(getTaskConfiguration());
this.name = configuration.getComponentName();
@@ -111,19 +178,6 @@ public abstract class AbstractStreamComponent<OUT> extends AbstractInvokable {
outSerializationDelegate.setInstance(outSerializer.createInstance());
}
- protected void setConfigOutputs(
- List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
- setSerializers();
- setCollector();
-
- int numberOfOutputs = configuration.getNumberOfOutputs();
- bufferTimeout = configuration.getBufferTimeout();
-
- for (int i = 0; i < numberOfOutputs; i++) {
- setPartitioner(i, outputs);
- }
- }
-
private void setPartitioner(int outputNumber,
List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
StreamPartitioner<OUT> outputPartitioner = null;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
index c06e664..a595cfa 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -20,19 +20,13 @@
package org.apache.flink.streaming.api.streamcomponent;
import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.MutableRecordReader;
import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -42,7 +36,6 @@ import org.apache.flink.util.MutableObjectIterator;
public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
AbstractStreamComponent<OUT> {
- private static final Log LOG = LogFactory.getLog(CoStreamTask.class);
protected StreamRecordSerializer<IN1> inputDeserializer1 = null;
protected StreamRecordSerializer<IN2> inputDeserializer2 = null;
@@ -52,13 +45,11 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
MutableObjectIterator<StreamRecord<IN1>> inputIter1;
MutableObjectIterator<StreamRecord<IN2>> inputIter2;
- private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private CoInvokable<IN1, IN2, OUT> userInvokable;
private static int numTasks;
public CoStreamTask() {
-
- outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+ outputHandler = new OutputHandler();
userInvokable = null;
numTasks = newComponent();
instanceID = numTasks;
@@ -94,7 +85,7 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
@Override
public void setInputsOutputs() {
- setConfigOutputs(outputs);
+ outputHandler.setConfigOutputs();
setConfigInputs();
inputIter1 = createInputIterator(inputs1, inputDeserializer1);
@@ -148,25 +139,7 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
@Override
public void invoke() throws Exception {
- if (LOG.isDebugEnabled()) {
- LOG.debug("TASK " + name + " invoked with instance id " + instanceID);
- }
-
- for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
- output.initializeSerializers();
- }
-
- userInvokable.open(getTaskConfiguration());
- userInvokable.invoke();
- userInvokable.close();
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("TASK " + name + " invoke finished with instance id " + instanceID);
- }
-
- for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
- output.flush();
- }
+ outputHandler.invokeUserFunction("CO-TASK", userInvokable);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
index 2514eb0..8f32dd7 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
@@ -19,8 +19,6 @@
package org.apache.flink.streaming.api.streamcomponent;
-import java.util.LinkedList;
-import java.util.List;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.TimeUnit;
@@ -37,7 +35,6 @@ public class StreamIterationSource<OUT extends Tuple> extends
private static final Log LOG = LogFactory.getLog(StreamIterationSource.class);
- private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private static int numSources;
private String iterationId;
@SuppressWarnings("rawtypes")
@@ -48,7 +45,7 @@ public class StreamIterationSource<OUT extends Tuple> extends
@SuppressWarnings("rawtypes")
public StreamIterationSource() {
- outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+ outputHandler = new OutputHandler();
numSources = newComponent();
instanceID = numSources;
dataChannel = new ArrayBlockingQueue<StreamRecord>(1);
@@ -57,9 +54,8 @@ public class StreamIterationSource<OUT extends Tuple> extends
@Override
public void setInputsOutputs() {
try {
- setConfigOutputs(outputs);
+ outputHandler.setConfigOutputs();
} catch (StreamComponentException e) {
- e.printStackTrace();
throw new StreamComponentException("Cannot register outputs", e);
}
@@ -82,9 +78,8 @@ public class StreamIterationSource<OUT extends Tuple> extends
LOG.debug("SOURCE " + name + " invoked with instance id " + instanceID);
}
- for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
- output.initializeSerializers();
- }
+ outputHandler.initializeOutputSerializers();
+
StreamRecord<OUT> nextRecord;
while (true) {
@@ -96,16 +91,13 @@ public class StreamIterationSource<OUT extends Tuple> extends
if (nextRecord == null) {
break;
}
- for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputHandler.getOutputs()) {
outSerializationDelegate.setInstance(nextRecord);
output.emit(outSerializationDelegate);
}
}
- for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
- output.flush();
- }
-
+ outputHandler.flushOutputs();
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
index a89935a..f695cb1 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamRecordWriter.java
@@ -12,7 +12,8 @@ import org.apache.flink.runtime.io.network.serialization.RecordSerializer;
import org.apache.flink.runtime.io.network.serialization.SpanningRecordSerializer;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-public class StreamRecordWriter<T extends IOReadableWritable> extends RecordWriter<T> {
+public class StreamRecordWriter<T extends IOReadableWritable> extends
+ RecordWriter<T> {
private final BufferProvider bufferPool;
@@ -31,12 +32,13 @@ public class StreamRecordWriter<T extends IOReadableWritable> extends RecordWrit
this(invokable, new RoundRobinChannelSelector<T>(), 1000);
}
- public StreamRecordWriter(AbstractInvokable invokable, ChannelSelector<T> channelSelector) {
+ public StreamRecordWriter(AbstractInvokable invokable,
+ ChannelSelector<T> channelSelector) {
this(invokable, channelSelector, 1000);
}
- public StreamRecordWriter(AbstractInvokable invokable, ChannelSelector<T> channelSelector,
- long timeout) {
+ public StreamRecordWriter(AbstractInvokable invokable,
+ ChannelSelector<T> channelSelector, long timeout) {
// initialize the gate
super(invokable);
@@ -60,20 +62,24 @@ public class StreamRecordWriter<T extends IOReadableWritable> extends RecordWrit
@Override
public void emit(final T record) throws IOException, InterruptedException {
- for (int targetChannel : this.channelSelector.selectChannels(record, this.numChannels)) {
+ for (int targetChannel : this.channelSelector.selectChannels(record,
+ this.numChannels)) {
// serialize with corresponding serializer and send full buffer
RecordSerializer<T> serializer = this.serializers[targetChannel];
synchronized (serializer) {
- RecordSerializer.SerializationResult result = serializer.addRecord(record);
+ RecordSerializer.SerializationResult result = serializer
+ .addRecord(record);
while (result.isFullBuffer()) {
Buffer buffer = serializer.getCurrentBuffer();
if (buffer != null) {
sendBuffer(buffer, targetChannel);
}
- buffer = this.bufferPool.requestBufferBlocking(this.bufferPool.getBufferSize());
+ buffer = this.bufferPool
+ .requestBufferBlocking(this.bufferPool
+ .getBufferSize());
result = serializer.setNextBuffer(buffer);
}
}
@@ -105,9 +111,8 @@ public class StreamRecordWriter<T extends IOReadableWritable> extends RecordWrit
Thread.sleep(timeout);
flush();
} catch (Exception e) {
- e.printStackTrace();
+ throw new RuntimeException(e);
}
-
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
index 6644d6f..494dfc1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
@@ -19,28 +19,16 @@
package org.apache.flink.streaming.api.streamcomponent;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.invokable.SourceInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamComponent<Tuple, OUT> {
- private static final Log LOG = LogFactory.getLog(StreamSource.class);
-
- private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private SourceInvokable<OUT> userInvokable;
private static int numSources;
public StreamSource() {
-
- outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+ outputHandler = new OutputHandler();
userInvokable = null;
numSources = newComponent();
instanceID = numSources;
@@ -49,7 +37,7 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
@Override
public void setInputsOutputs() {
try {
- setConfigOutputs(outputs);
+ outputHandler.setConfigOutputs();
} catch (StreamComponentException e) {
throw new StreamComponentException("Cannot register outputs for "
+ getClass().getSimpleName(), e);
@@ -59,31 +47,12 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
@Override
protected void setInvokable() {
userInvokable = getInvokable();
- // setCollector();
userInvokable.setCollector(collector);
}
@Override
public void invoke() throws Exception {
- if (LOG.isDebugEnabled()) {
- LOG.debug("SOURCE " + name + " invoked with instance id " + instanceID);
- }
-
- for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
- output.initializeSerializers();
- }
-
- userInvokable.open(getTaskConfiguration());
- userInvokable.invoke();
- userInvokable.close();
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("SOURCE " + name + " invoke finished with instance id " + instanceID);
- }
-
- for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
- output.flush();
- }
+ outputHandler.invokeUserFunction("SOURCE", userInvokable);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
index f809dae..32c10b6 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
@@ -19,28 +19,17 @@
package org.apache.flink.streaming.api.streamcomponent;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
SingleInputAbstractStreamComponent<IN, OUT> {
- private static final Log LOG = LogFactory.getLog(StreamTask.class);
-
- private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private StreamRecordInvokable<IN, OUT> userInvokable;
private static int numTasks;
public StreamTask() {
- outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+ outputHandler = new OutputHandler();
userInvokable = null;
numTasks = newComponent();
instanceID = numTasks;
@@ -49,7 +38,7 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
@Override
public void setInputsOutputs() {
setConfigInputs();
- setConfigOutputs(outputs);
+ outputHandler.setConfigOutputs();
inputIter = createInputIterator(inputs, inputSerializer);
}
@@ -57,29 +46,12 @@ public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
@Override
protected void setInvokable() {
userInvokable = getInvokable();
- userInvokable.initialize(collector, inputIter, inputSerializer, isMutable);
+ userInvokable.initialize(collector, inputIter, inputSerializer,
+ isMutable);
}
@Override
public void invoke() throws Exception {
- if (LOG.isDebugEnabled()) {
- LOG.debug("TASK " + name + " invoked with instance id " + instanceID);
- }
-
- for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
- output.initializeSerializers();
- }
-
- userInvokable.open(getTaskConfiguration());
- userInvokable.invoke();
- userInvokable.close();
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("TASK " + name + " invoke finished with instance id " + instanceID);
- }
-
- for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
- output.flush();
- }
+ outputHandler.invokeUserFunction("TASK", userInvokable);
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StateManager.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StateManager.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StateManager.java
index 46089f9..17d911a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StateManager.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StateManager.java
@@ -47,15 +47,12 @@ public class StateManager implements Runnable, Serializable {
ObjectInputStream ois = null;
try {
ois = new ObjectInputStream(new FileInputStream(filename));
- } catch (Exception e) {
- e.printStackTrace();
- }
- for (Object state : stateList) {
- try {
+ for (Object state : stateList) {
state = ois.readObject();
- } catch (Exception e) {
- e.printStackTrace();
}
+ ois.close();
+ } catch (Exception e) {
+ throw new RuntimeException(e);
}
}
@@ -67,7 +64,7 @@ public class StateManager implements Runnable, Serializable {
try {
oos = new ObjectOutputStream(new FileOutputStream(filename));
} catch (Exception e) {
- e.printStackTrace();
+ throw new RuntimeException(e);
}
// take snapshot of every registered state.
while (true) {
@@ -78,7 +75,7 @@ public class StateManager implements Runnable, Serializable {
oos.flush();
}
} catch (Exception e) {
- e.printStackTrace();
+ throw new RuntimeException(e);
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
index 3ea5b39..fc5978e 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
@@ -30,7 +30,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.jobgraph.JobGraph;
public class ClusterUtil {
- private static final Log log = LogFactory.getLog(ClusterUtil.class);
+ private static final Log LOG = LogFactory.getLog(ClusterUtil.class);
/**
* Executes the given JobGraph locally, on a NepheleMiniCluster
@@ -51,8 +51,8 @@ public class ClusterUtil {
exec.setNumTaskTracker(numberOfTaskTrackers);
Client client = new Client(new InetSocketAddress("localhost", 6498), configuration);
- if (log.isInfoEnabled()) {
- log.info("Running on mini cluster");
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Running on mini cluster");
}
try {
@@ -62,7 +62,7 @@ public class ClusterUtil {
exec.stop();
} catch (Exception e) {
- throw new RuntimeException(e.getMessage());
+ throw new RuntimeException(e);
}
}
@@ -71,8 +71,8 @@ public class ClusterUtil {
}
public static void runOnLocalCluster(JobGraph jobGraph, String IP, int port) {
- if (log.isInfoEnabled()) {
- log.info("Running on mini cluster");
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Running on mini cluster");
}
Configuration configuration = jobGraph.getJobConfiguration();
@@ -82,7 +82,7 @@ public class ClusterUtil {
try {
client.run(jobGraph, true);
} catch (ProgramInvocationException e) {
- throw new RuntimeException(e.getMessage());
+ throw new RuntimeException(e);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35cf874c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/TestDataUtil.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/TestDataUtil.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/TestDataUtil.java
index 44e0aa7..b9e3c4e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/TestDataUtil.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/TestDataUtil.java
@@ -39,7 +39,7 @@ public class TestDataUtil {
// TODO: Exception handling
// TODO: check checksum after download
- private static final Log log = LogFactory.getLog(TestDataUtil.class);
+ private static final Log LOG = LogFactory.getLog(TestDataUtil.class);
public static final String testDataDir = "src/test/resources/testdata/";
public static final String testRepoUrl = "http://info.ilab.sztaki.hu/~mbalassi/flink-streaming/testdata/";
public static final String testChekSumDir = "src/test/resources/testdata_checksum/";
@@ -67,8 +67,8 @@ public class TestDataUtil {
}
if (file.exists()) {
- if (log.isInfoEnabled()) {
- log.info(fileName + " already exists.");
+ if (LOG.isInfoEnabled()) {
+ LOG.info(fileName + " already exists.");
}
try {
@@ -78,16 +78,16 @@ public class TestDataUtil {
+ file.getAbsolutePath(), e);
}
if (!checkSumActaul.equals(checkSumDesired)) {
- if (log.isInfoEnabled()) {
- log.info("Checksum is incorrect.");
- log.info("Downloading file.");
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Checksum is incorrect.");
+ LOG.info("Downloading file.");
}
download(fileName);
}
} else {
- if (log.isInfoEnabled()) {
- log.info("File does not exist.");
- log.info("Downloading file.");
+ if (LOG.isInfoEnabled()) {
+ LOG.info("File does not exist.");
+ LOG.info("Downloading file.");
}
download(fileName);
}
@@ -95,7 +95,7 @@ public class TestDataUtil {
}
public static void download(String fileName) {
- log.info("downloading " + fileName);
+ LOG.info("downloading " + fileName);
try {
URL website = new URL(testRepoUrl + fileName);
[34/51] [abbrv] git commit: [streaming] Removed unnecessary files
from streaming project root
Posted by se...@apache.org.
[streaming] Removed unnecessary files from streaming project root
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/fed03a2f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/fed03a2f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/fed03a2f
Branch: refs/heads/master
Commit: fed03a2fe22ec33c81bcad0ad4ffb112d6277bbc
Parents: 1162cac
Author: Márton Balassi <mb...@ilab.sztaki.hu>
Authored: Wed Jul 30 12:30:51 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:22:50 2014 +0200
----------------------------------------------------------------------
flink-addons/flink-streaming/.travis.yml | 11 --
flink-addons/flink-streaming/LICENSE | 191 --------------------------
flink-addons/flink-streaming/README.md | 32 -----
3 files changed, 234 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fed03a2f/flink-addons/flink-streaming/.travis.yml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/.travis.yml b/flink-addons/flink-streaming/.travis.yml
deleted file mode 100644
index 99bb658..0000000
--- a/flink-addons/flink-streaming/.travis.yml
+++ /dev/null
@@ -1,11 +0,0 @@
-
-language: java
-jdk:
- - oraclejdk8
- - openjdk7
- - openjdk6
-
-
-
-script: "mvn -B clean verify"
-
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fed03a2f/flink-addons/flink-streaming/LICENSE
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/LICENSE b/flink-addons/flink-streaming/LICENSE
deleted file mode 100644
index 37ec93a..0000000
--- a/flink-addons/flink-streaming/LICENSE
+++ /dev/null
@@ -1,191 +0,0 @@
-Apache License
-Version 2.0, January 2004
-http://www.apache.org/licenses/
-
-TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-1. Definitions.
-
-"License" shall mean the terms and conditions for use, reproduction, and
-distribution as defined by Sections 1 through 9 of this document.
-
-"Licensor" shall mean the copyright owner or entity authorized by the copyright
-owner that is granting the License.
-
-"Legal Entity" shall mean the union of the acting entity and all other entities
-that control, are controlled by, or are under common control with that entity.
-For the purposes of this definition, "control" means (i) the power, direct or
-indirect, to cause the direction or management of such entity, whether by
-contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the
-outstanding shares, or (iii) beneficial ownership of such entity.
-
-"You" (or "Your") shall mean an individual or Legal Entity exercising
-permissions granted by this License.
-
-"Source" form shall mean the preferred form for making modifications, including
-but not limited to software source code, documentation source, and configuration
-files.
-
-"Object" form shall mean any form resulting from mechanical transformation or
-translation of a Source form, including but not limited to compiled object code,
-generated documentation, and conversions to other media types.
-
-"Work" shall mean the work of authorship, whether in Source or Object form, made
-available under the License, as indicated by a copyright notice that is included
-in or attached to the work (an example is provided in the Appendix below).
-
-"Derivative Works" shall mean any work, whether in Source or Object form, that
-is based on (or derived from) the Work and for which the editorial revisions,
-annotations, elaborations, or other modifications represent, as a whole, an
-original work of authorship. For the purposes of this License, Derivative Works
-shall not include works that remain separable from, or merely link (or bind by
-name) to the interfaces of, the Work and Derivative Works thereof.
-
-"Contribution" shall mean any work of authorship, including the original version
-of the Work and any modifications or additions to that Work or Derivative Works
-thereof, that is intentionally submitted to Licensor for inclusion in the Work
-by the copyright owner or by an individual or Legal Entity authorized to submit
-on behalf of the copyright owner. For the purposes of this definition,
-"submitted" means any form of electronic, verbal, or written communication sent
-to the Licensor or its representatives, including but not limited to
-communication on electronic mailing lists, source code control systems, and
-issue tracking systems that are managed by, or on behalf of, the Licensor for
-the purpose of discussing and improving the Work, but excluding communication
-that is conspicuously marked or otherwise designated in writing by the copyright
-owner as "Not a Contribution."
-
-"Contributor" shall mean Licensor and any individual or Legal Entity on behalf
-of whom a Contribution has been received by Licensor and subsequently
-incorporated within the Work.
-
-2. Grant of Copyright License.
-
-Subject to the terms and conditions of this License, each Contributor hereby
-grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free,
-irrevocable copyright license to reproduce, prepare Derivative Works of,
-publicly display, publicly perform, sublicense, and distribute the Work and such
-Derivative Works in Source or Object form.
-
-3. Grant of Patent License.
-
-Subject to the terms and conditions of this License, each Contributor hereby
-grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free,
-irrevocable (except as stated in this section) patent license to make, have
-made, use, offer to sell, sell, import, and otherwise transfer the Work, where
-such license applies only to those patent claims licensable by such Contributor
-that are necessarily infringed by their Contribution(s) alone or by combination
-of their Contribution(s) with the Work to which such Contribution(s) was
-submitted. If You institute patent litigation against any entity (including a
-cross-claim or counterclaim in a lawsuit) alleging that the Work or a
-Contribution incorporated within the Work constitutes direct or contributory
-patent infringement, then any patent licenses granted to You under this License
-for that Work shall terminate as of the date such litigation is filed.
-
-4. Redistribution.
-
-You may reproduce and distribute copies of the Work or Derivative Works thereof
-in any medium, with or without modifications, and in Source or Object form,
-provided that You meet the following conditions:
-
-You must give any other recipients of the Work or Derivative Works a copy of
-this License; and
-You must cause any modified files to carry prominent notices stating that You
-changed the files; and
-You must retain, in the Source form of any Derivative Works that You distribute,
-all copyright, patent, trademark, and attribution notices from the Source form
-of the Work, excluding those notices that do not pertain to any part of the
-Derivative Works; and
-If the Work includes a "NOTICE" text file as part of its distribution, then any
-Derivative Works that You distribute must include a readable copy of the
-attribution notices contained within such NOTICE file, excluding those notices
-that do not pertain to any part of the Derivative Works, in at least one of the
-following places: within a NOTICE text file distributed as part of the
-Derivative Works; within the Source form or documentation, if provided along
-with the Derivative Works; or, within a display generated by the Derivative
-Works, if and wherever such third-party notices normally appear. The contents of
-the NOTICE file are for informational purposes only and do not modify the
-License. You may add Your own attribution notices within Derivative Works that
-You distribute, alongside or as an addendum to the NOTICE text from the Work,
-provided that such additional attribution notices cannot be construed as
-modifying the License.
-You may add Your own copyright statement to Your modifications and may provide
-additional or different license terms and conditions for use, reproduction, or
-distribution of Your modifications, or for any such Derivative Works as a whole,
-provided Your use, reproduction, and distribution of the Work otherwise complies
-with the conditions stated in this License.
-
-5. Submission of Contributions.
-
-Unless You explicitly state otherwise, any Contribution intentionally submitted
-for inclusion in the Work by You to the Licensor shall be under the terms and
-conditions of this License, without any additional terms or conditions.
-Notwithstanding the above, nothing herein shall supersede or modify the terms of
-any separate license agreement you may have executed with Licensor regarding
-such Contributions.
-
-6. Trademarks.
-
-This License does not grant permission to use the trade names, trademarks,
-service marks, or product names of the Licensor, except as required for
-reasonable and customary use in describing the origin of the Work and
-reproducing the content of the NOTICE file.
-
-7. Disclaimer of Warranty.
-
-Unless required by applicable law or agreed to in writing, Licensor provides the
-Work (and each Contributor provides its Contributions) on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied,
-including, without limitation, any warranties or conditions of TITLE,
-NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A PARTICULAR PURPOSE. You are
-solely responsible for determining the appropriateness of using or
-redistributing the Work and assume any risks associated with Your exercise of
-permissions under this License.
-
-8. Limitation of Liability.
-
-In no event and under no legal theory, whether in tort (including negligence),
-contract, or otherwise, unless required by applicable law (such as deliberate
-and grossly negligent acts) or agreed to in writing, shall any Contributor be
-liable to You for damages, including any direct, indirect, special, incidental,
-or consequential damages of any character arising as a result of this License or
-out of the use or inability to use the Work (including but not limited to
-damages for loss of goodwill, work stoppage, computer failure or malfunction, or
-any and all other commercial damages or losses), even if such Contributor has
-been advised of the possibility of such damages.
-
-9. Accepting Warranty or Additional Liability.
-
-While redistributing the Work or Derivative Works thereof, You may choose to
-offer, and charge a fee for, acceptance of support, warranty, indemnity, or
-other liability obligations and/or rights consistent with this License. However,
-in accepting such obligations, You may act only on Your own behalf and on Your
-sole responsibility, not on behalf of any other Contributor, and only if You
-agree to indemnify, defend, and hold each Contributor harmless for any liability
-incurred by, or claims asserted against, such Contributor by reason of your
-accepting any such warranty or additional liability.
-
-END OF TERMS AND CONDITIONS
-
-APPENDIX: How to apply the Apache License to your work
-
-To apply the Apache License to your work, attach the following boilerplate
-notice, with the fields enclosed by brackets "[]" replaced with your own
-identifying information. (Don't include the brackets!) The text should be
-enclosed in the appropriate comment syntax for the file format. We also
-recommend that a file or class name and description of purpose be included on
-the same "printed page" as the copyright notice for easier identification within
-third-party archives.
-
- Copyright [yyyy] [name of copyright owner]
-
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fed03a2f/flink-addons/flink-streaming/README.md
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/README.md b/flink-addons/flink-streaming/README.md
deleted file mode 100644
index 20e655f..0000000
--- a/flink-addons/flink-streaming/README.md
+++ /dev/null
@@ -1,32 +0,0 @@
-# Apache Flink Streaming
-
-_"Big Data looks tiny from Stratosphere."_
-
-This repository implements stream data processing support for [Apache Flink](http://www.flink.incubator.apache.org) For more information please check ot the [Architecture Sketch](https://github.com/stratosphere/stratosphere-streaming/wiki/Architecture-Sketch).
-
-## Build From Source
-
-This tutorial shows how to build Apache Flink Streaming on your own system. Please open a bug report if you have any troubles!
-
-### Requirements
-* Unix-like environment (We use Linux, Mac OS X, Cygwin)
-* git
-* Maven (at least version 3.0.4)
-* Java 6 or 7
-
-### Get the source & Build it
-
-```
-git clone https://github.com/stratosphere/stratosphere-streaming.git
-cd stratosphere-streaming
-mvn clean assembly:assembly
-```
-
-### What to contribute
-* Bug reports
-* Bug fixes
-* Documentation
-* Tools that ease the use and development of Apache Flink
-* Well-written Apache Flink jobs
-
-Let us know if you have created a system that uses Apache Flink, so that we can link to you.
[41/51] [abbrv] git commit: [streaming] DataStream output naming
reworked from vertex to edge based model
Posted by se...@apache.org.
[streaming] DataStream output naming reworked from vertex to edge based model
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/9be98149
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/9be98149
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/9be98149
Branch: refs/heads/master
Commit: 9be9814972458aa1c67ccf6db5dd508244a89b21
Parents: d56d48f
Author: gyfora <gy...@gmail.com>
Authored: Tue Aug 5 23:15:13 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:23:40 2014 +0200
----------------------------------------------------------------------
.../flink/streaming/api/StreamConfig.java | 23 +++------
.../api/collector/DirectedStreamCollector.java | 29 +++++------
.../streaming/api/collector/OutputSelector.java | 8 +--
.../api/collector/StreamCollector.java | 42 +++++++++-------
.../streaming/api/datastream/CoDataStream.java | 21 ++------
.../api/datastream/ConnectedDataStream.java | 14 ------
.../streaming/api/datastream/DataStream.java | 44 +++--------------
.../api/datastream/IterativeDataStream.java | 34 +++++++------
.../datastream/SingleOutputStreamOperator.java | 18 ++++++-
.../api/datastream/SplitDataStream.java | 51 +++++++++++---------
.../streamcomponent/StreamIterationSource.java | 1 +
.../api/collector/DirectedOutputTest.java | 21 ++------
12 files changed, 133 insertions(+), 173 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index 7cfc808..6fe9878 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -40,7 +40,6 @@ public class StreamConfig {
private static final String NUMBER_OF_INPUTS = "numberOfInputs";
private static final String OUTPUT_NAME = "outputName_";
private static final String PARTITIONER_OBJECT = "partitionerObject_";
- private static final String USER_DEFINED_NAME = "userDefinedName";
private static final String NUMBER_OF_OUTPUT_CHANNELS = "numOfOutputs_";
private static final String ITERATION_ID = "iteration-id";
private static final String OUTPUT_SELECTOR = "outputSelector";
@@ -74,8 +73,7 @@ public class StreamConfig {
// CONFIGS
- public void setTypeWrapper(
- TypeSerializerWrapper<?, ?, ?> typeWrapper) {
+ public void setTypeWrapper(TypeSerializerWrapper<?, ?, ?> typeWrapper) {
config.setBytes("typeWrapper", SerializationUtils.serialize(typeWrapper));
}
@@ -166,12 +164,6 @@ public class StreamConfig {
return config.getString(FUNCTION_NAME, "");
}
- public void setUserDefinedName(List<String> userDefinedName) {
- if (!userDefinedName.isEmpty()) {
- config.setBytes(USER_DEFINED_NAME, SerializationUtils.serialize((Serializable) userDefinedName));
- }
- }
-
public void setDirectedEmit(boolean directedEmit) {
config.setBoolean(DIRECTED_EMIT, directedEmit);
}
@@ -212,28 +204,29 @@ public class StreamConfig {
return config.getInteger(NUMBER_OF_OUTPUT_CHANNELS + outputIndex, 0);
}
- public <T> void setPartitioner(int outputIndex,
- StreamPartitioner<T> partitionerObject) {
+ public <T> void setPartitioner(int outputIndex, StreamPartitioner<T> partitionerObject) {
config.setBytes(PARTITIONER_OBJECT + outputIndex,
SerializationUtils.serialize(partitionerObject));
}
- public <T> StreamPartitioner<T> getPartitioner(int outputIndex)
- throws ClassNotFoundException, IOException {
+ public <T> StreamPartitioner<T> getPartitioner(int outputIndex) throws ClassNotFoundException,
+ IOException {
return deserializeObject(config.getBytes(PARTITIONER_OBJECT + outputIndex,
SerializationUtils.serialize(new ShufflePartitioner<T>())));
}
public void setOutputName(int outputIndex, List<String> outputName) {
if (outputName != null) {
- config.setBytes(OUTPUT_NAME + outputIndex, SerializationUtils.serialize((Serializable) outputName));
+ config.setBytes(OUTPUT_NAME + outputIndex,
+ SerializationUtils.serialize((Serializable) outputName));
}
}
@SuppressWarnings("unchecked")
public List<String> getOutputName(int outputIndex) {
- return (List<String>) SerializationUtils.deserialize(config.getBytes(OUTPUT_NAME + outputIndex, null));
+ return (List<String>) SerializationUtils.deserialize(config.getBytes(OUTPUT_NAME
+ + outputIndex, null));
}
public void setNumberOfInputs(int numberOfInputs) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
index ced3de7..285a7b4 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
@@ -34,14 +34,14 @@ import org.apache.flink.util.StringUtils;
* A StreamCollector that uses user defined output names and a user defined
* output selector to make directed emits.
*
- * @param <T>
+ * @param <OUT>
* Type of the Tuple collected.
*/
-public class DirectedStreamCollector<T> extends StreamCollector<T> {
+public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
- OutputSelector<T> outputSelector;
+ OutputSelector<OUT> outputSelector;
private static final Log log = LogFactory.getLog(DirectedStreamCollector.class);
- private List<RecordWriter<SerializationDelegate<StreamRecord<T>>>> emitted;
+ private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> emitted;
/**
* Creates a new DirectedStreamCollector
@@ -54,11 +54,11 @@ public class DirectedStreamCollector<T> extends StreamCollector<T> {
* User defined {@link OutputSelector}
*/
public DirectedStreamCollector(int channelID,
- SerializationDelegate<StreamRecord<T>> serializationDelegate,
- OutputSelector<T> outputSelector) {
+ SerializationDelegate<StreamRecord<OUT>> serializationDelegate,
+ OutputSelector<OUT> outputSelector) {
super(channelID, serializationDelegate);
this.outputSelector = outputSelector;
- this.emitted = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<T>>>>();
+ this.emitted = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
}
@@ -70,7 +70,7 @@ public class DirectedStreamCollector<T> extends StreamCollector<T> {
* Object to be collected and emitted.
*/
@Override
- public void collect(T outputObject) {
+ public void collect(OUT outputObject) {
streamRecord.setObject(outputObject);
emit(streamRecord);
}
@@ -82,18 +82,19 @@ public class DirectedStreamCollector<T> extends StreamCollector<T> {
* @param streamRecord
* Record to emit.
*/
- private void emit(StreamRecord<T> streamRecord) {
+ private void emit(StreamRecord<OUT> streamRecord) {
Collection<String> outputNames = outputSelector.getOutputs(streamRecord.getObject());
streamRecord.setId(channelID);
serializationDelegate.setInstance(streamRecord);
emitted.clear();
for (String outputName : outputNames) {
try {
- RecordWriter<SerializationDelegate<StreamRecord<T>>> output = outputMap
- .get(outputName);
- if (!emitted.contains(output)) {
- output.emit(serializationDelegate);
- emitted.add(output);
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputMap
+ .get(outputName)) {
+ if (!emitted.contains(output)) {
+ output.emit(serializationDelegate);
+ emitted.add(output);
+ }
}
} catch (Exception e) {
if (log.isErrorEnabled()) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
index 17d7e7b..fbd2147 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java
@@ -31,10 +31,10 @@ import org.apache.flink.streaming.api.datastream.SplitDataStream;
* {@link SingleOutputStreamOperator#split} call. Every output object of a
* {@link SplitDataStream} will run through this operator to select outputs.
*
- * @param <T>
+ * @param <OUT>
* Type parameter of the split values.
*/
-public abstract class OutputSelector<T> implements Serializable {
+public abstract class OutputSelector<OUT> implements Serializable {
private static final long serialVersionUID = 1L;
private Collection<String> outputs;
@@ -43,7 +43,7 @@ public abstract class OutputSelector<T> implements Serializable {
outputs = new ArrayList<String>();
}
- Collection<String> getOutputs(T outputObject) {
+ Collection<String> getOutputs(OUT outputObject) {
outputs.clear();
select(outputObject, outputs);
return outputs;
@@ -60,5 +60,5 @@ public abstract class OutputSelector<T> implements Serializable {
* @param outputs
* Selected output names should be added to this collection.
*/
- public abstract void select(T value, Collection<String> outputs);
+ public abstract void select(OUT value, Collection<String> outputs);
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
index 20c3b78..fe21c29 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
@@ -33,22 +33,22 @@ import org.apache.flink.util.Collector;
import org.apache.flink.util.StringUtils;
/**
- * Collector for tuples in Apache Flink stream processing. The collected
- * values will be wrapped with ID in a {@link StreamRecord} and then
- * emitted to the outputs.
+ * Collector for tuples in Apache Flink stream processing. The collected values
+ * will be wrapped with ID in a {@link StreamRecord} and then emitted to the
+ * outputs.
*
- * @param <T>
+ * @param <OUT>
* Type of the Tuples/Objects collected.
*/
-public class StreamCollector<T> implements Collector<T> {
+public class StreamCollector<OUT> implements Collector<OUT> {
private static final Log LOG = LogFactory.getLog(StreamCollector.class);
- protected StreamRecord<T> streamRecord;
+ protected StreamRecord<OUT> streamRecord;
protected int channelID;
- private List<RecordWriter<SerializationDelegate<StreamRecord<T>>>> outputs;
- protected Map<String, RecordWriter<SerializationDelegate<StreamRecord<T>>>> outputMap;
- protected SerializationDelegate<StreamRecord<T>> serializationDelegate;
+ private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
+ protected Map<String, List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>> outputMap;
+ protected SerializationDelegate<StreamRecord<OUT>> serializationDelegate;
/**
* Creates a new StreamCollector
@@ -59,13 +59,13 @@ public class StreamCollector<T> implements Collector<T> {
* Serialization delegate used for serialization
*/
public StreamCollector(int channelID,
- SerializationDelegate<StreamRecord<T>> serializationDelegate) {
+ SerializationDelegate<StreamRecord<OUT>> serializationDelegate) {
this.serializationDelegate = serializationDelegate;
- this.streamRecord = new StreamRecord<T>();
+ this.streamRecord = new StreamRecord<OUT>();
this.channelID = channelID;
- this.outputs = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<T>>>>();
- this.outputMap = new HashMap<String, RecordWriter<SerializationDelegate<StreamRecord<T>>>>();
+ this.outputs = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+ this.outputMap = new HashMap<String, List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>>();
}
/**
@@ -76,13 +76,19 @@ public class StreamCollector<T> implements Collector<T> {
* @param outputNames
* User defined names of the output.
*/
- public void addOutput(RecordWriter<SerializationDelegate<StreamRecord<T>>> output,
+ public void addOutput(RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output,
List<String> outputNames) {
outputs.add(output);
for (String outputName : outputNames) {
if (outputName != null) {
if (!outputMap.containsKey(outputName)) {
- outputMap.put(outputName, output);
+ outputMap.put(outputName,
+ new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>());
+ outputMap.get(outputName).add(output);
+ } else {
+ if (!outputMap.get(outputName).contains(output)) {
+ outputMap.get(outputName).add(output);
+ }
}
}
@@ -97,7 +103,7 @@ public class StreamCollector<T> implements Collector<T> {
* Object to be collected and emitted.
*/
@Override
- public void collect(T outputObject) {
+ public void collect(OUT outputObject) {
streamRecord.setObject(outputObject);
emit(streamRecord);
}
@@ -108,10 +114,10 @@ public class StreamCollector<T> implements Collector<T> {
* @param streamRecord
* StreamRecord to emit.
*/
- private void emit(StreamRecord<T> streamRecord) {
+ private void emit(StreamRecord<OUT> streamRecord) {
streamRecord.setId(channelID);
serializationDelegate.setInstance(streamRecord);
- for (RecordWriter<SerializationDelegate<StreamRecord<T>>> output : outputs) {
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
try {
output.emit(serializationDelegate);
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
index c6cb8af..b974b1d 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoDataStream.java
@@ -78,11 +78,11 @@ public class CoDataStream<IN1, IN2> {
/**
* Applies a CoMap transformation on two separate {@link DataStream}s. The
- * transformation calls a {@link CoMapFunction#map1} for each element
- * of the first input and {@link CoMapFunction#map2} for each element
- * of the second input. Each CoMapFunction call returns exactly one element.
- * The user can also extend {@link RichCoMapFunction} to gain access to
- * other features provided by the {@link RichFuntion} interface.
+ * transformation calls a {@link CoMapFunction#map1} for each element of the
+ * first input and {@link CoMapFunction#map2} for each element of the second
+ * input. Each CoMapFunction call returns exactly one element. The user can
+ * also extend {@link RichCoMapFunction} to gain access to other features
+ * provided by the {@link RichFuntion} interface.
*
* @param coMapper
* The CoMapFunction used to jointly transform the two input
@@ -113,17 +113,6 @@ public class CoDataStream<IN1, IN2> {
input1.connectGraph(input1, returnStream.getId(), 1);
input1.connectGraph(input2, returnStream.getId(), 2);
- if ((input1.userDefinedName != null) && (input2.userDefinedName != null)) {
- throw new RuntimeException("An operator cannot have two names");
- } else {
- if (input1.userDefinedName != null) {
- returnStream.name(input1.getUserDefinedNames());
- }
-
- if (input2.userDefinedName != null) {
- returnStream.name(input2.getUserDefinedNames());
- }
- }
// TODO consider iteration
return returnStream;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
index d17990c..1d8fb48 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
@@ -62,24 +62,10 @@ public class ConnectedDataStream<OUT> extends DataStream<OUT> {
// }
protected void addConnection(DataStream<OUT> stream) {
- if ((stream.userDefinedName != null) || (this.userDefinedName != null)) {
- if (!this.userDefinedName.equals(stream.userDefinedName)) {
- throw new RuntimeException("Error: Connected NamedDataStreams must have same names");
- }
- }
connectedStreams.add(stream.copy());
}
@Override
- protected List<String> getUserDefinedNames() {
- List<String> nameList = new ArrayList<String>();
- for (DataStream<OUT> stream : connectedStreams) {
- nameList.add(stream.userDefinedName);
- }
- return nameList;
- }
-
- @Override
protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
ConnectedDataStream<OUT> returnStream = (ConnectedDataStream<OUT>) this.copy();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index b692984..d15eaa5 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -36,7 +36,6 @@ import org.apache.flink.api.java.functions.RichGroupReduceFunction;
import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.JobGraphBuilder;
-import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.PrintSinkFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
@@ -80,7 +79,7 @@ public abstract class DataStream<OUT> {
protected final StreamExecutionEnvironment environment;
protected final String id;
protected int degreeOfParallelism;
- protected String userDefinedName;
+ protected List<String> userDefinedNames;
protected StreamPartitioner<OUT> partitioner;
protected final JobGraphBuilder jobGraphBuilder;
@@ -105,6 +104,7 @@ public abstract class DataStream<OUT> {
this.environment = environment;
this.degreeOfParallelism = environment.getDegreeOfParallelism();
this.jobGraphBuilder = environment.getJobGraphBuilder();
+ this.userDefinedNames = new ArrayList<String>();
this.partitioner = new ForwardPartitioner<OUT>();
}
@@ -119,7 +119,7 @@ public abstract class DataStream<OUT> {
this.environment = dataStream.environment;
this.id = dataStream.id;
this.degreeOfParallelism = dataStream.degreeOfParallelism;
- this.userDefinedName = dataStream.userDefinedName;
+ this.userDefinedNames = new ArrayList<String>(dataStream.userDefinedNames);
this.partitioner = dataStream.partitioner;
this.jobGraphBuilder = dataStream.jobGraphBuilder;
@@ -734,36 +734,9 @@ public abstract class DataStream<OUT> {
.toString());
}
- if (userDefinedName != null) {
- returnStream.name(getUserDefinedNames());
- }
-
return returnStream;
}
- protected List<String> getUserDefinedNames() {
- List<String> nameList = new ArrayList<String>();
- nameList.add(userDefinedName);
- return nameList;
- }
-
- /**
- * Gives the data transformation(vertex) a user defined name in order to use
- * with directed outputs. The {@link OutputSelector} of the input vertex
- * should use this name for directed emits.
- *
- * @param name
- * The name to set
- * @return The named DataStream.
- */
- protected DataStream<OUT> name(List<String> name) {
-
- userDefinedName = name.get(0);
- jobGraphBuilder.setUserDefinedName(id, name);
-
- return this;
- }
-
/**
* Internal function for setting the partitioner for the DataStream
*
@@ -795,11 +768,12 @@ public abstract class DataStream<OUT> {
protected <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
if (inputStream instanceof ConnectedDataStream) {
for (DataStream<X> stream : ((ConnectedDataStream<X>) inputStream).connectedStreams) {
- jobGraphBuilder.setEdge(stream.getId(), outputID, stream.partitioner, typeNumber);
+ jobGraphBuilder.setEdge(stream.getId(), outputID, stream.partitioner, typeNumber,
+ inputStream.userDefinedNames);
}
} else {
jobGraphBuilder.setEdge(inputStream.getId(), outputID, inputStream.partitioner,
- typeNumber);
+ typeNumber, inputStream.userDefinedNames);
}
}
@@ -834,11 +808,7 @@ public abstract class DataStream<OUT> {
throw new RuntimeException("Cannot serialize SinkFunction");
}
- inputStream.connectGraph(inputStream, returnStream.getId(), 0);
-
- if (this.copy().userDefinedName != null) {
- returnStream.name(getUserDefinedNames());
- }
+ inputStream.connectGraph(inputStream.copy(), returnStream.getId(), 0);
return returnStream;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
index b9aadcd..bdadee4 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
@@ -19,27 +19,30 @@
package org.apache.flink.streaming.api.datastream;
+import java.util.Arrays;
+import java.util.List;
+
import org.apache.flink.streaming.partitioner.ForwardPartitioner;
/**
* The iterative data stream represents the start of an iteration in a
* {@link DataStream}.
*
- * @param <T>
+ * @param <IN>
* Type of the DataStream
*/
-public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, IterativeDataStream<T>> {
+public class IterativeDataStream<IN> extends SingleOutputStreamOperator<IN, IterativeDataStream<IN>> {
static Integer iterationCount = 0;
protected Integer iterationID;
- protected IterativeDataStream(DataStream<T> dataStream) {
+ protected IterativeDataStream(DataStream<IN> dataStream) {
super(dataStream);
iterationID = iterationCount;
iterationCount++;
}
- protected IterativeDataStream(DataStream<T> dataStream, Integer iterationID) {
+ protected IterativeDataStream(DataStream<IN> dataStream, Integer iterationID) {
super(dataStream);
this.iterationID = iterationID;
}
@@ -55,8 +58,8 @@ public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, Iterat
* The data stream that can be fed back to the next iteration.
*
*/
- public DataStream<T> closeWith(DataStream<T> iterationResult) {
- return closeWith(iterationResult, null);
+ public DataStream<IN> closeWith(DataStream<IN> iterationResult) {
+ return closeWith(iterationResult, "iterate");
}
/**
@@ -73,31 +76,34 @@ public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, Iterat
* when used with directed emits
*
*/
- public <R> DataStream<T> closeWith(DataStream<T> iterationTail, String iterationName) {
+ public <R> DataStream<IN> closeWith(DataStream<IN> iterationTail, String iterationName) {
DataStream<R> returnStream = new DataStreamSink<R>(environment, "iterationSink");
jobGraphBuilder.addIterationSink(returnStream.getId(), iterationTail.getId(),
- iterationID.toString(), iterationTail.getParallelism(), iterationName);
+ iterationID.toString(), iterationTail.getParallelism());
jobGraphBuilder.setIterationSourceParallelism(iterationID.toString(),
iterationTail.getParallelism());
+ List<String> name = Arrays.asList(new String[] { iterationName });
+
if (iterationTail instanceof ConnectedDataStream) {
- for (DataStream<T> stream : ((ConnectedDataStream<T>) iterationTail).connectedStreams) {
+ for (DataStream<IN> stream : ((ConnectedDataStream<IN>) iterationTail).connectedStreams) {
String inputID = stream.getId();
- jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<T>(),
- 0);
+ jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<IN>(),
+ 0, name);
}
} else {
+
jobGraphBuilder.setEdge(iterationTail.getId(), returnStream.getId(),
- new ForwardPartitioner<T>(), 0);
+ new ForwardPartitioner<IN>(), 0, name);
}
return iterationTail;
}
@Override
- protected IterativeDataStream<T> copy() {
- return new IterativeDataStream<T>(this, iterationID);
+ protected IterativeDataStream<IN> copy() {
+ return new IterativeDataStream<IN>(this, iterationID);
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index 9af4dc8..f798563 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -105,6 +105,22 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
* @return The {@link SplitDataStream}
*/
public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector) {
+ return split(outputSelector, null);
+ }
+
+ /**
+ * Operator used for directing tuples to specific named outputs using an
+ * {@link OutputSelector}. Calling this method on an operator creates a new
+ * {@link SplitDataStream}.
+ *
+ * @param outputSelector
+ * The user defined {@link OutputSelector} for directing the
+ * tuples.
+ * @param outputNames
+ * An array of all the output names to be used for selectAll
+ * @return The {@link SplitDataStream}
+ */
+ public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector, String[] outputNames) {
try {
jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
@@ -112,7 +128,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
throw new RuntimeException("Cannot serialize OutputSelector");
}
- return new SplitDataStream<OUT>(this);
+ return new SplitDataStream<OUT>(this, outputNames);
}
@SuppressWarnings("unchecked")
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
index 8bcde44..69d8f61 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
@@ -19,20 +19,26 @@
package org.apache.flink.streaming.api.datastream;
+import java.util.Arrays;
+
+import org.apache.flink.streaming.api.collector.OutputSelector;
+
/**
* The SplitDataStream represents an operator that has been split using an
* {@link OutputSelector}. Named outputs can be selected using the
* {@link #select} function.
*
- * @param <T>
+ * @param <OUT>
* The type of the output.
*/
-public class SplitDataStream<T> {
+public class SplitDataStream<OUT> {
- DataStream<T> dataStream;
+ DataStream<OUT> dataStream;
+ String[] allNames;
- protected SplitDataStream(DataStream<T> dataStream) {
+ protected SplitDataStream(DataStream<OUT> dataStream, String[] outputNames) {
this.dataStream = dataStream.copy();
+ this.allNames = outputNames;
}
/**
@@ -41,29 +47,30 @@ public class SplitDataStream<T> {
* @param outputNames
* The output names for which the operator will receive the
* input.
- * @return Returns the modified DataStream
+ * @return Returns the selected DataStream
*/
- public DataStream<T> select(String... outputNames) {
- DataStream<T> returnStream = selectOutput(outputNames[0]);
- for (int i = 1; i < outputNames.length; i++) {
- if (outputNames[i] == "") {
- throw new IllegalArgumentException("User defined name must not be empty string");
- }
-
- returnStream = connectWithNames(returnStream, selectOutput(outputNames[i]));
- }
- return returnStream;
+ public DataStream<OUT> select(String... outputNames) {
+ return selectOutput(outputNames);
}
- private DataStream<T> connectWithNames(DataStream<T> stream1, DataStream<T> stream2) {
- ConnectedDataStream<T> returnStream = new ConnectedDataStream<T>(stream1.copy());
- returnStream.connectedStreams.add(stream2.copy());
- return returnStream;
+ /**
+ * Selects all output names from a split data stream. Output names must
+ * predefined to use selectAll.
+ *
+ * @return Returns the selected DataStream
+ */
+ public DataStream<OUT> selectAll() {
+ if (allNames != null) {
+ return selectOutput(allNames);
+ } else {
+ throw new RuntimeException(
+ "Output names must be predefined in order to use select all.");
+ }
}
- private DataStream<T> selectOutput(String outputName) {
- DataStream<T> returnStream = dataStream.copy();
- returnStream.userDefinedName = outputName;
+ private DataStream<OUT> selectOutput(String[] outputName) {
+ DataStream<OUT> returnStream = dataStream.copy();
+ returnStream.userDefinedNames = Arrays.asList(outputName);
return returnStream;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
index cf3d47e..67d5066 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
@@ -58,6 +58,7 @@ public class StreamIterationSource<OUT extends Tuple> extends
setConfigOutputs(outputs);
setSinkSerializer();
} catch (StreamComponentException e) {
+ e.printStackTrace();
throw new StreamComponentException("Cannot register outputs", e);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9be98149/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index 08387f9..0e390e4 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -20,7 +20,6 @@
package org.apache.flink.streaming.api.collector;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
import java.util.Arrays;
import java.util.Collection;
@@ -96,9 +95,11 @@ public class DirectedOutputTest {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
- SplitDataStream<Long> s = env.generateSequence(1, 6).split(new MySelector());
+ SplitDataStream<Long> s = env.generateSequence(1, 6).split(new MySelector(),
+ new String[] { "ds1", "ds2" });
DataStream<Long> ds1 = s.select("ds1").shuffle().map(new PlusTwo()).addSink(new EvenSink());
DataStream<Long> ds2 = s.select("ds2").map(new PlusTwo()).addSink(new OddSink());
+
env.executeTest(32);
HashSet<Long> expectedEven = new HashSet<Long>(Arrays.asList(4L, 6L, 8L));
@@ -107,20 +108,4 @@ public class DirectedOutputTest {
assertEquals(expectedEven, evenSet);
assertEquals(expectedOdd, oddSet);
}
-
- @SuppressWarnings({ "unchecked" })
- @Test
- public void directNamingTest() {
- LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
-
- LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
- SplitDataStream<Long> s = env.generateSequence(1, 10).split(new MySelector());
- try {
- s.select("ds2").connectWith(s.select("ds1"));
- fail();
- } catch (Exception e) {
- // Exception thrown
- }
-
- }
}
[21/51] [abbrv] git commit: [streaming] Refactored StreamComponents
Posted by se...@apache.org.
[streaming] Refactored StreamComponents
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/d282eef1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/d282eef1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/d282eef1
Branch: refs/heads/master
Commit: d282eef1a316125debc9a85af9e6b909d319eb18
Parents: 330d8fd
Author: ghermann <re...@gmail.com>
Authored: Mon Jul 28 08:21:30 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:21:13 2014 +0200
----------------------------------------------------------------------
.../flink/streaming/api/StreamConfig.java | 2 +-
.../AbstractStreamComponent.java | 66 +++++++++++++++++---
.../api/streamcomponent/CoStreamTask.java | 30 ++-------
.../SingleInputAbstractStreamComponent.java | 55 ++++++----------
.../streamcomponent/StreamIterationSink.java | 15 ++---
.../streamcomponent/StreamIterationSource.java | 15 ++---
.../api/streamcomponent/StreamSink.java | 26 ++------
.../api/streamcomponent/StreamSource.java | 23 ++-----
.../api/streamcomponent/StreamTask.java | 33 ++--------
9 files changed, 109 insertions(+), 156 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d282eef1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index d677046..fc4a1dd 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -105,7 +105,7 @@ public class StreamConfig {
}
}
- public StreamComponentInvokable getUserInvokableObject() {
+ public <T extends StreamComponentInvokable> T getUserInvokableObject() {
try {
return deserializeObject(config.getBytes(SERIALIZEDUDF, null));
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d282eef1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index 775e722..22c079c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -26,6 +26,10 @@ import org.apache.commons.lang.SerializationUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.FilterFunction;
+import org.apache.flink.api.java.functions.FlatMapFunction;
+import org.apache.flink.api.java.functions.GroupReduceFunction;
+import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
@@ -39,7 +43,9 @@ import org.apache.flink.streaming.api.StreamConfig;
import org.apache.flink.streaming.api.collector.DirectedStreamCollector;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.collector.StreamCollector;
+import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
+import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.partitioner.StreamPartitioner;
@@ -60,15 +66,21 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
protected String name;
private static int numComponents = 0;
protected boolean isMutable;
-
+ protected Object function;
+ protected String functionName;
+
protected static int newComponent() {
numComponents++;
return numComponents;
}
protected void initialize() {
- configuration = new StreamConfig(getTaskConfiguration());
- name = configuration.getComponentName();
+ this.configuration = new StreamConfig(getTaskConfiguration());
+ this.name = configuration.getComponentName();
+ this.isMutable = configuration.getMutability();
+ this.functionName = configuration.getFunctionName();
+ this.function = configuration.getFunction();
+
}
protected Collector<OUT> setCollector() {
@@ -83,6 +95,35 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
return collector;
}
+ protected void setSerializers() {
+ try {
+ if (functionName.equals("flatMap")) {
+ setSerializer(function, FlatMapFunction.class, 1);
+ } else if (functionName.equals("map")) {
+ setSerializer(function, MapFunction.class, 1);
+ } else if (functionName.equals("batchReduce")) {
+ setSerializer(function, GroupReduceFunction.class, 1);
+ } else if (functionName.equals("filter")) {
+ setSerializer(function, FilterFunction.class, 0);
+ } else if (functionName.equals("source")) {
+ setSerializer(function, UserSourceInvokable.class, 0);
+ } else if (functionName.equals("coMap")) {
+ setSerializer(function, CoMapFunction.class, 2);
+ } else if (functionName.equals("elements")) {
+ outTupleTypeInfo = new TupleTypeInfo<OUT>(TypeExtractor.getForObject(function));
+
+ outTupleSerializer = new StreamRecordSerializer<OUT>(outTupleTypeInfo.createSerializer());
+ outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(
+ outTupleSerializer);
+ } else {
+ throw new Exception("Wrong operator name: " + functionName);
+ }
+ } catch (Exception e) {
+ throw new StreamComponentException(e);
+ }
+
+ }
+
@SuppressWarnings({ "rawtypes", "unchecked" })
protected void setSerializer(Object function, Class<?> clazz, int typeParameter) {
outTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
@@ -94,7 +135,9 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
protected void setConfigOutputs(
List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
-
+ setSerializers();
+ setCollector();
+
int numberOfOutputs = configuration.getNumberOfOutputs();
for (int i = 0; i < numberOfOutputs; i++) {
@@ -146,10 +189,7 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
* Class of the invokable function
* @return The StreamComponent object
*/
- protected StreamComponentInvokable getInvokable(
- Class<? extends StreamComponentInvokable> userFunctionClass) {
-
- this.isMutable = configuration.getMutability();
+ protected <T extends StreamComponentInvokable> T getInvokable() {
return configuration.getUserInvokableObject();
}
@@ -170,6 +210,16 @@ public abstract class AbstractStreamComponent<OUT extends Tuple> extends Abstrac
return (T) SerializationUtils.deserialize(serializedObject);
}
+
+ @Override
+ public void registerInputOutput() {
+ initialize();
+ setInputsOutputs();
+ setInvokable();
+ }
+
+ protected abstract void setInputsOutputs();
+
protected abstract void setInvokable();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d282eef1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
index 6a9c897..4d531f7 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -55,7 +55,6 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private CoInvokable<IN1, IN2, OUT> userFunction;
-// private int[] numberOfOutputChannels;
private static int numTasks;
public CoStreamTask() {
@@ -84,7 +83,6 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
@SuppressWarnings({ "unchecked", "rawtypes" })
private void setDeserializers(Object function, Class<? extends AbstractFunction> clazz) {
-
TupleTypeInfo<IN1> inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz,
function.getClass(), 0, null, null);
inTupleSerializer1 = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
@@ -95,37 +93,17 @@ public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tupl
}
@Override
- public void registerInputOutput() {
- initialize();
-
- setSerializers();
- setCollector();
-
- // inputs1 = setConfigInputs();
+ public void setInputsOutputs() {
+ setConfigOutputs(outputs);
setConfigInputs();
inputIter1 = createInputIterator(inputs1, inTupleSerializer1);
-
- // inputs2 = setConfigInputs();
inputIter2 = createInputIterator(inputs2, inTupleSerializer2);
-
- setConfigOutputs(outputs);
-
-// numberOfOutputChannels = new int[outputs.size()];
-// for (int i = 0; i < numberOfOutputChannels.length; i++) {
-// numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
-// }
-
- setInvokable();
}
-
- @SuppressWarnings({ "rawtypes", "unchecked" })
+
@Override
protected void setInvokable() {
- // Default value is a CoMapInvokable
- Class<? extends CoInvokable> userFunctionClass = configuration.getUserInvokableClass();
-
- userFunction = (CoInvokable<IN1, IN2, OUT>) getInvokable(userFunctionClass);
+ userFunction = getInvokable();
userFunction.initialize(collector, inputIter1, inTupleSerializer1, inputIter2,
inTupleSerializer2, isMutable);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d282eef1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
index 86b26ff..0b5b377 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
@@ -31,47 +31,35 @@ import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.MutableRecordReader;
import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+import org.apache.flink.util.MutableObjectIterator;
public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT extends Tuple> extends
AbstractStreamComponent<OUT> {
protected StreamRecordSerializer<IN> inTupleSerializer = null;
+ protected MutableObjectIterator<StreamRecord<IN>> inputIter;
+ protected MutableReader<IOReadableWritable> inputs;
- protected void setSerializers() {
- String operatorName = configuration.getFunctionName();
-
- Object function = configuration.getFunction();
+ protected void setDeserializers() {
try {
- if (operatorName.equals("flatMap")) {
- setSerializerDeserializer(function, FlatMapFunction.class);
- } else if (operatorName.equals("map")) {
- setSerializerDeserializer(function, MapFunction.class);
- } else if (operatorName.equals("batchReduce")) {
- setSerializerDeserializer(function, GroupReduceFunction.class);
- } else if (operatorName.equals("filter")) {
+ if (functionName.equals("flatMap")) {
+ setDeserializer(function, FlatMapFunction.class);
+ } else if (functionName.equals("map")) {
+ setDeserializer(function, MapFunction.class);
+ } else if (functionName.equals("batchReduce")) {
+ setDeserializer(function, GroupReduceFunction.class);
+ } else if (functionName.equals("filter")) {
setDeserializer(function, FilterFunction.class);
- setSerializer(function, FilterFunction.class, 0);
- } else if (operatorName.equals("sink")) {
- setDeserializer(function, SinkFunction.class);
- } else if (operatorName.equals("source")) {
+ } else if (functionName.equals("source")) {
setSerializer(function, UserSourceInvokable.class, 0);
- } else if (operatorName.equals("coMap")) {
- setSerializer(function, CoMapFunction.class, 2);
- //setDeserializers(function, CoMapFunction.class);
- } else if (operatorName.equals("elements")) {
- outTupleTypeInfo = new TupleTypeInfo<OUT>(TypeExtractor.getForObject(function));
-
- outTupleSerializer = new StreamRecordSerializer<OUT>(outTupleTypeInfo.createSerializer());
- outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(
- outTupleSerializer);
+ } else if (functionName.equals("sink")) {
+ setDeserializer(function, SinkFunction.class);
} else {
- throw new Exception("Wrong operator name: " + operatorName);
+ throw new Exception("Wrong operator name: " + functionName);
}
} catch (Exception e) {
@@ -79,11 +67,6 @@ public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT e
}
}
- private void setSerializerDeserializer(Object function, Class<? extends AbstractFunction> clazz) {
- setDeserializer(function, clazz);
- setSerializer(function, clazz, 1);
- }
-
@SuppressWarnings({ "unchecked", "rawtypes" })
private void setDeserializer(Object function, Class<? extends AbstractFunction> clazz) {
TupleTypeInfo<IN> inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
@@ -102,12 +85,14 @@ public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT e
}
@SuppressWarnings("unchecked")
- protected MutableReader<IOReadableWritable> getConfigInputs() throws StreamComponentException {
+ protected void setConfigInputs() throws StreamComponentException {
+ setDeserializers();
+
int numberOfInputs = configuration.getNumberOfInputs();
if (numberOfInputs < 2) {
- return new MutableRecordReader<IOReadableWritable>(this);
+ inputs = new MutableRecordReader<IOReadableWritable>(this);
} else {
MutableRecordReader<IOReadableWritable>[] recordReaders = (MutableRecordReader<IOReadableWritable>[]) new MutableRecordReader<?>[numberOfInputs];
@@ -115,7 +100,7 @@ public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT e
for (int i = 0; i < numberOfInputs; i++) {
recordReaders[i] = new MutableRecordReader<IOReadableWritable>(this);
}
- return new MutableUnionRecordReader<IOReadableWritable>(recordReaders);
+ inputs = new MutableUnionRecordReader<IOReadableWritable>(recordReaders);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d282eef1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
index 224fdfb..5532626 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
@@ -25,7 +25,6 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.util.MutableObjectIterator;
import org.apache.flink.util.StringUtils;
@@ -34,8 +33,6 @@ public class StreamIterationSink<IN extends Tuple> extends SingleInputAbstractSt
private static final Log LOG = LogFactory.getLog(StreamIterationSink.class);
- @SuppressWarnings("rawtypes")
- private MutableReader inputs;
MutableObjectIterator<StreamRecord<IN>> inputIter;
private String iterationId;
@SuppressWarnings("rawtypes")
@@ -45,22 +42,22 @@ public class StreamIterationSink<IN extends Tuple> extends SingleInputAbstractSt
}
@Override
- public void registerInputOutput() {
- initialize();
-
+ public void setInputsOutputs() {
try {
- setSerializers();
+ setConfigInputs();
setSinkSerializer();
- inputs = getConfigInputs();
+
inputIter = createInputIterator(inputs, inTupleSerializer);
+
iterationId = configuration.getIterationId();
dataChannel = BlockingQueueBroker.instance().get(iterationId);
+
} catch (Exception e) {
throw new StreamComponentException(String.format(
"Cannot register inputs of StreamIterationSink %s", iterationId), e);
}
}
-
+
@Override
public void invoke() throws Exception {
if (LOG.isDebugEnabled()) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d282eef1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
index 37e8e0f..d020058 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
@@ -38,7 +38,6 @@ public class StreamIterationSource<OUT extends Tuple> extends SingleInputAbstrac
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private static int numSources;
-// private int[] numberOfOutputChannels;
private String iterationId;
@SuppressWarnings("rawtypes")
private BlockingQueue<StreamRecord> dataChannel;
@@ -53,29 +52,23 @@ public class StreamIterationSource<OUT extends Tuple> extends SingleInputAbstrac
}
@Override
- public void registerInputOutput() {
- initialize();
-
+ public void setInputsOutputs() {
try {
- setSerializers();
setConfigOutputs(outputs);
+ setSinkSerializer();
} catch (StreamComponentException e) {
throw new StreamComponentException("Cannot register outputs", e);
}
-// numberOfOutputChannels = new int[outputs.size()];
-// for (int i = 0; i < numberOfOutputChannels.length; i++) {
-// numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
-// }
-
iterationId = configuration.getIterationId();
try {
BlockingQueueBroker.instance().handIn(iterationId, dataChannel);
} catch (Exception e) {
}
- }
+ }
+
@Override
public void invoke() throws Exception {
if (LOG.isDebugEnabled()) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d282eef1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
index 8cbe0ea..7ac117e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
@@ -22,19 +22,12 @@ package org.apache.flink.streaming.api.streamcomponent;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.io.network.api.MutableReader;
-import org.apache.flink.streaming.api.invokable.SinkInvokable;
import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.util.MutableObjectIterator;
public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamComponent<IN, IN> {
private static final Log LOG = LogFactory.getLog(StreamSink.class);
- @SuppressWarnings("rawtypes")
- private MutableReader inputs;
- private MutableObjectIterator<StreamRecord<IN>> inputIter;
private StreamRecordInvokable<IN, IN> userFunction;
public StreamSink() {
@@ -42,28 +35,21 @@ public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamCompo
}
@Override
- public void registerInputOutput() {
- initialize();
-
+ public void setInputsOutputs() {
try {
- setSerializers();
+ setConfigInputs();
setSinkSerializer();
- inputs = getConfigInputs();
+
inputIter = createInputIterator(inputs, inTupleSerializer);
} catch (Exception e) {
throw new StreamComponentException("Cannot register inputs for "
+ getClass().getSimpleName(), e);
}
-
- setInvokable();
}
-
- @SuppressWarnings({ "rawtypes", "unchecked" })
+
@Override
- protected void setInvokable() {
- Class<? extends SinkInvokable> userFunctionClass = configuration.getUserInvokableClass();
-
- userFunction = (SinkInvokable<IN>) getInvokable(userFunctionClass);
+ protected void setInvokable() {
+ userFunction = getInvokable();
userFunction.initialize(collector, inputIter, inTupleSerializer, isMutable);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d282eef1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
index c4f38ce..856c917 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
@@ -37,7 +37,6 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private UserSourceInvokable<OUT> userFunction;
private static int numSources;
-// private int[] numberOfOutputChannels;
public StreamSource() {
@@ -48,33 +47,19 @@ public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamCo
}
@Override
- public void registerInputOutput() {
- initialize();
-
+ public void setInputsOutputs() {
try {
- setSerializers();
- setCollector();
setConfigOutputs(outputs);
} catch (StreamComponentException e) {
throw new StreamComponentException("Cannot register outputs for "
+ getClass().getSimpleName(), e);
- }
-
-// numberOfOutputChannels = new int[outputs.size()];
-// for (int i = 0; i < numberOfOutputChannels.length; i++) {
-// numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
-// }
-
- setInvokable();
+ }
}
-
- @SuppressWarnings({ "rawtypes", "unchecked" })
+
@Override
protected void setInvokable() {
// Default value is a TaskInvokable even if it was called from a source
- Class<? extends UserSourceInvokable> userFunctionClass = configuration.getUserInvokableClass();
-// .getClass("userfunction", UserSourceInvokable.class, UserSourceInvokable.class);
- userFunction = (UserSourceInvokable<OUT>) getInvokable(userFunctionClass);
+ userFunction = getInvokable();
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d282eef1/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
index 12e064f..19b1c4b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
@@ -25,60 +25,39 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.runtime.io.network.api.MutableReader;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
-import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.util.MutableObjectIterator;
public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
SingleInputAbstractStreamComponent<IN, OUT> {
private static final Log LOG = LogFactory.getLog(StreamTask.class);
- private MutableReader<IOReadableWritable> inputs;
- MutableObjectIterator<StreamRecord<IN>> inputIter;
private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
private StreamRecordInvokable<IN, OUT> userFunction;
-// private int[] numberOfOutputChannels;
private static int numTasks;
public StreamTask() {
-
outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
userFunction = null;
numTasks = newComponent();
instanceID = numTasks;
}
-
+
@Override
- public void registerInputOutput() {
- initialize();
-
- setSerializers();
- setCollector();
- inputs = getConfigInputs();
+ public void setInputsOutputs() {
+ setConfigInputs();
setConfigOutputs(outputs);
- inputIter = createInputIterator(inputs, inTupleSerializer);
-
-// numberOfOutputChannels = new int[outputs.size()];
-// for (int i = 0; i < numberOfOutputChannels.length; i++) {
-// numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
-// }
-
- setInvokable();
+ inputIter = createInputIterator(inputs, inTupleSerializer);
}
-
- @SuppressWarnings({ "rawtypes", "unchecked" })
+
@Override
protected void setInvokable() {
// Default value is a TaskInvokable even if it was called from a source
- Class<? extends UserTaskInvokable> userFunctionClass = configuration.getUserInvokableClass();
- userFunction = (UserTaskInvokable<IN, OUT>) getInvokable(userFunctionClass);
+ userFunction = getInvokable();
userFunction.initialize(collector, inputIter, inTupleSerializer, isMutable);
}
[08/51] [abbrv] git commit: [streaming] Added CoFunctions for two
type inputs
Posted by se...@apache.org.
[streaming] Added CoFunctions for two type inputs
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/f436690f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/f436690f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/f436690f
Branch: refs/heads/master
Commit: f436690fb4ec4d9b5dc7342b5124b6fb0e855d76
Parents: 6e52195
Author: ghermann <re...@gmail.com>
Authored: Tue Jul 22 11:28:26 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Aug 18 16:19:14 2014 +0200
----------------------------------------------------------------------
.../apache/flink/streaming/api/DataStream.java | 7 +
.../flink/streaming/api/JobGraphBuilder.java | 95 ++++++---
.../api/StreamExecutionEnvironment.java | 43 +++-
.../api/function/co/CoMapFunction.java | 29 +++
.../api/invokable/operator/co/CoInvokable.java | 55 ++++++
.../invokable/operator/co/CoMapInvokable.java | 53 +++++
.../AbstractStreamComponent.java | 144 +++-----------
.../api/streamcomponent/CoStreamTask.java | 198 +++++++++++++++++++
.../SingleInputAbstractStreamComponent.java | 129 ++++++++++++
.../streamcomponent/StreamIterationSink.java | 2 +-
.../streamcomponent/StreamIterationSource.java | 2 +-
.../api/streamcomponent/StreamSink.java | 2 +-
.../api/streamcomponent/StreamSource.java | 2 +-
.../api/streamcomponent/StreamTask.java | 2 +-
.../api/invokable/operator/CoMapTest.java | 77 ++++++++
15 files changed, 675 insertions(+), 165 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
index 4356795..6ef2faf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/DataStream.java
@@ -29,6 +29,7 @@ import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.api.StreamExecutionEnvironment.ConnectionType;
import org.apache.flink.streaming.api.collector.OutputSelector;
+import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
import org.apache.flink.streaming.api.function.sink.WriteFormatAsText;
@@ -36,6 +37,7 @@ import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
/**
* A DataStream represents a stream of elements of the same type. A DataStream
@@ -387,6 +389,11 @@ public class DataStream<T extends Tuple> {
new BatchReduceInvokable<T, R>(reducer, batchSize));
}
+ public <T2 extends Tuple, R extends Tuple> DataStream<R> coMapWith(CoMapFunction<T, T2, R> coMapper, DataStream<T2> otherStream) {
+ return environment.addCoFunction("coMap", new DataStream<T>(this), new DataStream<T2>(otherStream), coMapper, new CoMapInvokable<T, T2, R>(coMapper));
+ }
+
+
/**
* Applies a reduce transformation on preset "time" chunks of the
* DataStream. The transformation calls a {@link GroupReduceFunction} on
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index 2b3ee5a..022fcd8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -41,12 +41,13 @@ import org.apache.flink.runtime.jobgraph.JobInputVertex;
import org.apache.flink.runtime.jobgraph.JobOutputVertex;
import org.apache.flink.runtime.jobgraph.JobTaskVertex;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.operators.util.TaskConfig;
import org.apache.flink.streaming.api.collector.OutputSelector;
-import org.apache.flink.streaming.api.invokable.SinkInvokable;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
+import org.apache.flink.streaming.api.invokable.UserSinkInvokable;
import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
+import org.apache.flink.streaming.api.streamcomponent.CoStreamTask;
import org.apache.flink.streaming.api.streamcomponent.StreamIterationSink;
import org.apache.flink.streaming.api.streamcomponent.StreamIterationSource;
import org.apache.flink.streaming.api.streamcomponent.StreamSink;
@@ -70,8 +71,8 @@ public class JobGraphBuilder {
// Graph attributes
private Map<String, AbstractJobVertex> components;
private Map<String, Integer> componentParallelism;
- private Map<String, Boolean> mutability;
- private Map<String, List<String>> outEdgeList;
+ private Map<String, ArrayList<String>> outEdgeList;
+ private Map<String, ArrayList<Integer>> outEdgeType;
private Map<String, List<String>> inEdgeList;
private Map<String, List<StreamPartitioner<? extends Tuple>>> connectionTypes;
private Map<String, String> userDefinedNames;
@@ -101,8 +102,8 @@ public class JobGraphBuilder {
components = new HashMap<String, AbstractJobVertex>();
componentParallelism = new HashMap<String, Integer>();
- mutability = new HashMap<String, Boolean>();
- outEdgeList = new HashMap<String, List<String>>();
+ outEdgeList = new HashMap<String, ArrayList<String>>();
+ outEdgeType = new HashMap<String, ArrayList<Integer>>();
inEdgeList = new HashMap<String, List<String>>();
connectionTypes = new HashMap<String, List<StreamPartitioner<? extends Tuple>>>();
userDefinedNames = new HashMap<String, String>();
@@ -171,7 +172,7 @@ public class JobGraphBuilder {
setBytesFrom(iterationHead, componentName);
setEdge(componentName, iterationHead,
- connectionTypes.get(inEdgeList.get(iterationHead).get(0)).get(0));
+ connectionTypes.get(inEdgeList.get(iterationHead).get(0)).get(0), 0);
if (LOG.isDebugEnabled()) {
LOG.debug("ITERATION SOURCE: " + componentName);
@@ -204,6 +205,18 @@ public class JobGraphBuilder {
}
}
+ public <IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> void addCoTask(
+ String componentName, CoInvokable<IN1, IN2, OUT> taskInvokableObject,
+ String operatorName, byte[] serializedFunction, int parallelism) {
+
+ addComponent(componentName, CoStreamTask.class, taskInvokableObject, operatorName,
+ serializedFunction, parallelism);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("CO-TASK: " + componentName);
+ }
+ }
+
/**
* Adds sink to the JobGraph with the given parameters
*
@@ -218,7 +231,7 @@ public class JobGraphBuilder {
* @param parallelism
* Number of parallel instances created
*/
- public void addSink(String componentName, SinkInvokable<? extends Tuple> InvokableObject,
+ public void addSink(String componentName, UserSinkInvokable<? extends Tuple> InvokableObject,
String operatorName, byte[] serializedFunction, int parallelism) {
addComponent(componentName, StreamSink.class, InvokableObject, operatorName,
@@ -288,11 +301,11 @@ public class JobGraphBuilder {
componentClasses.put(componentName, componentClass);
setParallelism(componentName, parallelism);
- mutability.put(componentName, false);
invokableObjects.put(componentName, invokableObject);
operatorNames.put(componentName, operatorName);
serializedFunctions.put(componentName, serializedFunction);
outEdgeList.put(componentName, new ArrayList<String>());
+ outEdgeType.put(componentName, new ArrayList<Integer>());
inEdgeList.put(componentName, new ArrayList<String>());
connectionTypes.put(componentName, new ArrayList<StreamPartitioner<? extends Tuple>>());
iterationTailCount.put(componentName, 0);
@@ -321,7 +334,8 @@ public class JobGraphBuilder {
if (componentClass.equals(StreamSource.class)
|| componentClass.equals(StreamIterationSource.class)) {
component = new JobInputVertex(componentName, this.jobGraph);
- } else if (componentClass.equals(StreamTask.class)) {
+ } else if (componentClass.equals(StreamTask.class)
+ || componentClass.equals(CoStreamTask.class)) {
component = new JobTaskVertex(componentName, this.jobGraph);
} else if (componentClass.equals(StreamSink.class)
|| componentClass.equals(StreamIterationSink.class)) {
@@ -334,9 +348,7 @@ public class JobGraphBuilder {
LOG.debug("Parallelism set: " + parallelism + " for " + componentName);
}
- Configuration config = new TaskConfig(component.getConfiguration()).getConfiguration();
-
- config.setBoolean("isMutable", mutability.get(componentName));
+ Configuration config = component.getConfiguration();
// Set vertex config
if (invokableObject != null) {
@@ -428,10 +440,6 @@ public class JobGraphBuilder {
componentParallelism.put(componentName, parallelism);
}
- public void setMutability(String componentName, boolean isMutable) {
- mutability.put(componentName, isMutable);
- }
-
/**
* Connects two vertices in the JobGraph using the selected partitioner
* settings
@@ -442,10 +450,13 @@ public class JobGraphBuilder {
* Name of the downstream(input) vertex
* @param partitionerObject
* Partitioner object
+ * @param typeNumber
+ * Number of the type (used at co-functions)
*/
public void setEdge(String upStreamComponentName, String downStreamComponentName,
- StreamPartitioner<? extends Tuple> partitionerObject) {
+ StreamPartitioner<? extends Tuple> partitionerObject, int typeNumber) {
outEdgeList.get(upStreamComponentName).add(downStreamComponentName);
+ outEdgeType.get(upStreamComponentName).add(typeNumber);
inEdgeList.get(downStreamComponentName).add(upStreamComponentName);
connectionTypes.get(upStreamComponentName).add(partitionerObject);
}
@@ -463,10 +474,13 @@ public class JobGraphBuilder {
* @param downStreamComponentName
* Name of the downstream component, that will receive the
* records
+ * @param typeNumber
+ * Number of the type (used at co-functions)
*/
public <T extends Tuple> void broadcastConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName) {
- setEdge(upStreamComponentName, downStreamComponentName, new BroadcastPartitioner<T>());
+ String upStreamComponentName, String downStreamComponentName, int typeNumber) {
+ setEdge(upStreamComponentName, downStreamComponentName, new BroadcastPartitioner<T>(),
+ typeNumber);
}
/**
@@ -485,12 +499,15 @@ public class JobGraphBuilder {
* records
* @param keyPosition
* Position of key in the tuple
+ * @param typeNumber
+ * Number of the type (used at co-functions)
*/
public <T extends Tuple> void fieldsConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName, int keyPosition) {
+ String upStreamComponentName, String downStreamComponentName, int keyPosition,
+ int typeNumber) {
setEdge(upStreamComponentName, downStreamComponentName, new FieldsPartitioner<T>(
- keyPosition));
+ keyPosition), typeNumber);
}
/**
@@ -505,10 +522,13 @@ public class JobGraphBuilder {
* Name of the upstream component, that will emit the tuples
* @param downStreamComponentName
* Name of the downstream component, that will receive the tuples
+ * @param typeNumber
+ * Number of the type (used at co-functions)
*/
public <T extends Tuple> void globalConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName) {
- setEdge(upStreamComponentName, downStreamComponentName, new GlobalPartitioner<T>());
+ String upStreamComponentName, String downStreamComponentName, int typeNumber) {
+ setEdge(upStreamComponentName, downStreamComponentName, new GlobalPartitioner<T>(),
+ typeNumber);
}
/**
@@ -523,10 +543,13 @@ public class JobGraphBuilder {
* Name of the upstream component, that will emit the tuples
* @param downStreamComponentName
* Name of the downstream component, that will receive the tuples
+ * @param typeNumber
+ * Number of the type (used at co-functions)
*/
public <T extends Tuple> void shuffleConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName) {
- setEdge(upStreamComponentName, downStreamComponentName, new ShufflePartitioner<T>());
+ String upStreamComponentName, String downStreamComponentName, int typeNumber) {
+ setEdge(upStreamComponentName, downStreamComponentName, new ShufflePartitioner<T>(),
+ typeNumber);
}
/**
@@ -542,10 +565,13 @@ public class JobGraphBuilder {
* Name of the upstream component, that will emit the tuples
* @param downStreamComponentName
* Name of the downstream component, that will receive the tuples
+ * @param typeNumber
+ * Number of the type (used at co-functions)
*/
public <T extends Tuple> void forwardConnect(DataStream<T> inputStream,
- String upStreamComponentName, String downStreamComponentName) {
- setEdge(upStreamComponentName, downStreamComponentName, new ForwardPartitioner<T>());
+ String upStreamComponentName, String downStreamComponentName, int typeNumber) {
+ setEdge(upStreamComponentName, downStreamComponentName, new ForwardPartitioner<T>(),
+ typeNumber);
}
/**
@@ -565,8 +591,7 @@ public class JobGraphBuilder {
AbstractJobVertex upStreamComponent = components.get(upStreamComponentName);
AbstractJobVertex downStreamComponent = components.get(downStreamComponentName);
- Configuration config = new TaskConfig(upStreamComponent.getConfiguration())
- .getConfiguration();
+ Configuration config = upStreamComponent.getConfiguration();
try {
if (partitionerObject.getClass().equals(ForwardPartitioner.class)) {
@@ -729,15 +754,21 @@ public class JobGraphBuilder {
for (String componentName : outEdgeList.keySet()) {
createVertex(componentName);
}
-
+ int inputNumber = 0;
for (String upStreamComponentName : outEdgeList.keySet()) {
+
int i = 0;
+ ArrayList<Integer> outEdgeTypeList = outEdgeType.get(upStreamComponentName);
+
for (String downStreamComponentName : outEdgeList.get(upStreamComponentName)) {
+ Configuration downStreamComponentConfig = components.get(downStreamComponentName)
+ .getConfiguration();
+ downStreamComponentConfig.setInteger("inputType_" + inputNumber++, outEdgeTypeList.get(i));
+
connect(upStreamComponentName, downStreamComponentName,
connectionTypes.get(upStreamComponentName).get(i));
i++;
}
-
}
setAutomaticInstanceSharing();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
index c1eca4a..9b1a16a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamExecutionEnvironment.java
@@ -42,6 +42,7 @@ import org.apache.flink.streaming.api.function.source.GenSequenceFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.api.invokable.SinkInvokable;
import org.apache.flink.streaming.api.invokable.UserTaskInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
/**
* {@link ExecutionEnvironment} for streaming jobs. An instance of it is
@@ -310,7 +311,7 @@ public abstract class StreamExecutionEnvironment {
throw new RuntimeException("Cannot serialize user defined function");
}
- connectGraph(inputStream, returnStream.getId());
+ connectGraph(inputStream, returnStream.getId(), 0);
if (inputStream.iterationflag) {
returnStream.addIterationSource(inputStream.iterationID.toString());
@@ -319,6 +320,30 @@ public abstract class StreamExecutionEnvironment {
return returnStream;
}
+
+ protected <T1 extends Tuple, T2 extends Tuple, R extends Tuple> DataStream<R> addCoFunction(String functionName, DataStream<T1> inputStream1, DataStream<T2> inputStream2, final AbstractFunction function,
+ CoInvokable<T1, T2, R> functionInvokable) {
+
+ DataStream<R> returnStream = new DataStream<R>(this, functionName);
+
+ try {
+ jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable, functionName,
+ SerializationUtils.serialize(function), degreeOfParallelism);
+ } catch (SerializationException e) {
+ throw new RuntimeException("Cannot serialize user defined function");
+ }
+
+ connectGraph(inputStream1, returnStream.getId(), 1);
+ connectGraph(inputStream2, returnStream.getId(), 2);
+
+ // TODO consider iteration
+// if (inputStream.iterationflag) {
+// returnStream.addIterationSource(inputStream.iterationID.toString());
+// inputStream.iterationflag = false;
+// }
+
+ return returnStream;
+ }
protected <T extends Tuple, R extends Tuple> void addIterationSource(DataStream<T> inputStream,
String iterationID) {
@@ -340,7 +365,7 @@ public abstract class StreamExecutionEnvironment {
for (int i = 0; i < inputStream.connectIDs.size(); i++) {
String input = inputStream.connectIDs.get(i);
- jobGraphBuilder.forwardConnect(inputStream, input, returnStream.getId());
+ jobGraphBuilder.forwardConnect(inputStream, input, returnStream.getId(), 0);
}
}
@@ -368,7 +393,7 @@ public abstract class StreamExecutionEnvironment {
throw new RuntimeException("Cannot serialize SinkFunction");
}
- connectGraph(inputStream, returnStream.getId());
+ connectGraph(inputStream, returnStream.getId(), 0);
return returnStream;
}
@@ -523,8 +548,10 @@ public abstract class StreamExecutionEnvironment {
* ID of the output
* @param <T>
* type of the input stream
+ * @param typeNumber
+ * Number of the type (used at co-functions)
*/
- private <T extends Tuple> void connectGraph(DataStream<T> inputStream, String outputID) {
+ private <T extends Tuple> void connectGraph(DataStream<T> inputStream, String outputID, int typeNumber) {
for (int i = 0; i < inputStream.connectIDs.size(); i++) {
ConnectionType type = inputStream.ctypes.get(i);
@@ -533,16 +560,16 @@ public abstract class StreamExecutionEnvironment {
switch (type) {
case SHUFFLE:
- jobGraphBuilder.shuffleConnect(inputStream, input, outputID);
+ jobGraphBuilder.shuffleConnect(inputStream, input, outputID, typeNumber);
break;
case BROADCAST:
- jobGraphBuilder.broadcastConnect(inputStream, input, outputID);
+ jobGraphBuilder.broadcastConnect(inputStream, input, outputID, typeNumber);
break;
case FIELD:
- jobGraphBuilder.fieldsConnect(inputStream, input, outputID, param);
+ jobGraphBuilder.fieldsConnect(inputStream, input, outputID, param, typeNumber);
break;
case FORWARD:
- jobGraphBuilder.forwardConnect(inputStream, input, outputID);
+ jobGraphBuilder.forwardConnect(inputStream, input, outputID, typeNumber);
break;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
new file mode 100644
index 0000000..5885cbf
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoMapFunction.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.function.co;
+
+import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+
+public abstract class CoMapFunction<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends AbstractFunction {
+ private static final long serialVersionUID = 1L;
+
+ public abstract OUT map1(IN1 value);
+ public abstract OUT map2(IN2 value);
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
new file mode 100644
index 0000000..733b61e
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
@@ -0,0 +1,55 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable.operator.co;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+
+public abstract class CoInvokable<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
+ StreamComponentInvokable {
+
+ private static final long serialVersionUID = 1L;
+
+ protected Collector<OUT> collector;
+ protected MutableObjectIterator<StreamRecord<IN1>> recordIterator1;
+ protected MutableObjectIterator<StreamRecord<IN2>> recordIterator2;
+ protected StreamRecord<IN1> reuse1;
+ protected StreamRecord<IN2> reuse2;
+
+ public void initialize(Collector<OUT> collector,
+ MutableObjectIterator<StreamRecord<IN1>> recordIterator1,
+ StreamRecordSerializer<IN1> serializer1,
+ MutableObjectIterator<StreamRecord<IN2>> recordIterator2,
+ StreamRecordSerializer<IN2> serializer2) {
+ this.collector = collector;
+
+ this.recordIterator1 = recordIterator1;
+ this.reuse1 = serializer1.createInstance();
+
+ this.recordIterator2 = recordIterator2;
+ this.reuse2 = serializer2.createInstance();
+ }
+
+ public abstract void invoke() throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
new file mode 100644
index 0000000..4a30425
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoMapInvokable.java
@@ -0,0 +1,53 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable.operator.co;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.function.co.CoMapFunction;
+
+public class CoMapInvokable<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
+ CoInvokable<IN1, IN2, OUT> {
+ private static final long serialVersionUID = 1L;
+
+ private CoMapFunction<IN1, IN2, OUT> mapper;
+
+ public CoMapInvokable(CoMapFunction<IN1, IN2, OUT> mapper) {
+ this.mapper = mapper;
+ }
+
+ // TODO rework this as UnionRecordReader
+ @Override
+ public void invoke() throws Exception {
+ boolean noMoreRecordOnInput1;
+ boolean noMoreRecordOnInput2;
+
+ do {
+ noMoreRecordOnInput1 = recordIterator1.next(reuse1) == null;
+ if (!noMoreRecordOnInput1) {
+ collector.collect(mapper.map1(reuse1.getTuple()));
+ }
+
+ noMoreRecordOnInput2 = recordIterator2.next(reuse2) == null;
+ if (!noMoreRecordOnInput2) {
+ collector.collect(mapper.map2(reuse2.getTuple()));
+ }
+ } while (!noMoreRecordOnInput1 && !noMoreRecordOnInput2);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
index f90caf8..1a51492 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
@@ -19,28 +19,18 @@
package org.apache.flink.streaming.api.streamcomponent;
-import java.io.ByteArrayInputStream;
import java.io.IOException;
-import java.io.ObjectInputStream;
import java.util.List;
import org.apache.commons.lang.SerializationUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.flink.api.common.functions.AbstractFunction;
import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.FilterFunction;
-import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.functions.GroupReduceFunction;
-import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.MutableReader;
-import org.apache.flink.runtime.io.network.api.MutableRecordReader;
-import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.operators.util.ReaderIterator;
@@ -49,9 +39,7 @@ import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.collector.DirectedStreamCollector;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.collector.StreamCollector;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.invokable.StreamComponentInvokable;
-import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.partitioner.ShufflePartitioner;
@@ -59,14 +47,10 @@ import org.apache.flink.streaming.partitioner.StreamPartitioner;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
-public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tuple> extends
- AbstractInvokable {
+public abstract class AbstractStreamComponent<OUT extends Tuple> extends AbstractInvokable {
+
private static final Log LOG = LogFactory.getLog(AbstractStreamComponent.class);
- protected TupleTypeInfo<IN> inTupleTypeInfo = null;
- protected StreamRecordSerializer<IN> inTupleSerializer = null;
- protected DeserializationDelegate<StreamRecord<IN>> inDeserializationDelegate = null;
-
protected TupleTypeInfo<OUT> outTupleTypeInfo = null;
protected StreamRecordSerializer<OUT> outTupleSerializer = null;
protected SerializationDelegate<StreamRecord<OUT>> outSerializationDelegate = null;
@@ -105,109 +89,17 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
}
return collector;
}
-
- protected void setSerializers() {
- byte[] operatorBytes = configuration.getBytes("operator", null);
- String operatorName = configuration.getString("operatorName", "");
-
- Object function = null;
- try {
- ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(operatorBytes));
- function = in.readObject();
-
- if (operatorName.equals("flatMap")) {
- setSerializerDeserializer(function, FlatMapFunction.class);
- } else if (operatorName.equals("map")) {
- setSerializerDeserializer(function, MapFunction.class);
- } else if (operatorName.equals("batchReduce")) {
- setSerializerDeserializer(function, GroupReduceFunction.class);
- } else if (operatorName.equals("filter")) {
- setDeserializer(function, FilterFunction.class);
- setSerializer(function, FilterFunction.class, 0);
- } else if (operatorName.equals("sink")) {
- setDeserializer(function, SinkFunction.class);
- } else if (operatorName.equals("source")) {
- setSerializer(function, UserSourceInvokable.class, 0);
- } else if (operatorName.equals("elements")) {
- outTupleTypeInfo = new TupleTypeInfo<OUT>(TypeExtractor.getForObject(function));
-
- outTupleSerializer = new StreamRecordSerializer<OUT>(outTupleTypeInfo.createSerializer());
- outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(
- outTupleSerializer);
- } else {
- throw new Exception("Wrong operator name!");
- }
-
- } catch (Exception e) {
- throw new StreamComponentException(e);
- // throw new StreamComponentException("Nonsupported object (named "
- // + operatorName
- // + ") passed as operator");
- }
- }
-
- private void setSerializerDeserializer(Object function, Class<? extends AbstractFunction> clazz) {
- setDeserializer(function, clazz);
- setSerializer(function, clazz, 1);
- }
-
- @SuppressWarnings({ "unchecked", "rawtypes" })
- private void setDeserializer(Object function, Class<? extends AbstractFunction> clazz) {
- inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
- 0, null, null);
-
- inTupleSerializer = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
- inDeserializationDelegate = new DeserializationDelegate<StreamRecord<IN>>(inTupleSerializer);
- }
+
@SuppressWarnings({ "rawtypes", "unchecked" })
- private void setSerializer(Object function, Class<?> clazz, int typeParameter) {
+ protected void setSerializer(Object function, Class<?> clazz, int typeParameter) {
outTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
typeParameter, null, null);
outTupleSerializer = new StreamRecordSerializer(outTupleTypeInfo.createSerializer());
outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(outTupleSerializer);
}
-
- @SuppressWarnings("unchecked")
- protected void setSinkSerializer() {
- if (outSerializationDelegate != null) {
- inTupleTypeInfo = (TupleTypeInfo<IN>) outTupleTypeInfo;
-
- inTupleSerializer = new StreamRecordSerializer<IN>(inTupleTypeInfo.createSerializer());
- inDeserializationDelegate = new DeserializationDelegate<StreamRecord<IN>>(inTupleSerializer);
- }
- }
-
- @SuppressWarnings("unchecked")
- protected MutableReader<IOReadableWritable> getConfigInputs() throws StreamComponentException {
- int numberOfInputs = configuration.getInteger("numberOfInputs", 0);
-
- if (numberOfInputs < 2) {
-
- return new MutableRecordReader<IOReadableWritable>(this);
-
- } else {
- MutableRecordReader<IOReadableWritable>[] recordReaders = (MutableRecordReader<IOReadableWritable>[]) new MutableRecordReader<?>[numberOfInputs];
-
- for (int i = 0; i < numberOfInputs; i++) {
- recordReaders[i] = new MutableRecordReader<IOReadableWritable>(this);
- }
- return new MutableUnionRecordReader<IOReadableWritable>(recordReaders);
- }
- }
-
- protected MutableObjectIterator<StreamRecord<IN>> createInputIterator(MutableReader<?> inputReader,
- TypeSerializer<?> serializer) {
-
- // generic data type serialization
- @SuppressWarnings("unchecked")
- MutableReader<DeserializationDelegate<?>> reader = (MutableReader<DeserializationDelegate<?>>) inputReader;
- @SuppressWarnings({ "unchecked", "rawtypes" })
- final MutableObjectIterator<StreamRecord<IN>> iter = new ReaderIterator(reader, serializer);
- return iter;
-
- }
+
protected void setConfigOutputs(List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
@@ -217,7 +109,7 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
setPartitioner(i, outputs);
}
}
-
+
private void setPartitioner(int outputNumber,
List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
@@ -242,12 +134,12 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
+ " with " + outputNumber + " outputs");
}
} catch (Exception e) {
- throw new StreamComponentException("Cannot deserialize partitioner "
- + outputPartitioner.getClass().getSimpleName() + " with " + outputNumber
- + " outputs");
+ throw new StreamComponentException("Cannot deserialize " + outputPartitioner.getClass().getSimpleName() + " of " +
+ name + " with " + outputNumber
+ + " outputs", e);
}
}
-
+
/**
* Reads and creates a StreamComponent from the config.
*
@@ -274,12 +166,24 @@ public abstract class AbstractStreamComponent<IN extends Tuple, OUT extends Tupl
return userFunction;
}
+
+ protected <IN extends Tuple> MutableObjectIterator<StreamRecord<IN>> createInputIterator(MutableReader<?> inputReader,
+ TypeSerializer<?> serializer) {
+ // generic data type serialization
+ @SuppressWarnings("unchecked")
+ MutableReader<DeserializationDelegate<?>> reader = (MutableReader<DeserializationDelegate<?>>) inputReader;
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ final MutableObjectIterator<StreamRecord<IN>> iter = new ReaderIterator(reader, serializer);
+ return iter;
+ }
+
@SuppressWarnings("unchecked")
- private static <T> T deserializeObject(byte[] serializedObject) throws IOException,
+ protected static <T> T deserializeObject(byte[] serializedObject) throws IOException,
ClassNotFoundException {
return (T) SerializationUtils.deserialize(serializedObject);
}
-
+
protected abstract void setInvokable();
+
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
new file mode 100644
index 0000000..4c1cf42
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
@@ -0,0 +1,198 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.streamcomponent;
+
+import java.io.ByteArrayInputStream;
+import java.io.ObjectInputStream;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.runtime.io.network.api.MutableReader;
+import org.apache.flink.runtime.io.network.api.MutableRecordReader;
+import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
+import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.function.co.CoMapFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+import org.apache.flink.util.MutableObjectIterator;
+
+public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
+ AbstractStreamComponent<OUT> {
+ private static final Log LOG = LogFactory.getLog(CoStreamTask.class);
+
+ protected StreamRecordSerializer<IN1> inTupleSerializer1 = null;
+ protected StreamRecordSerializer<IN2> inTupleSerializer2 = null;
+
+ private MutableReader<IOReadableWritable> inputs1;
+ private MutableReader<IOReadableWritable> inputs2;
+ MutableObjectIterator<StreamRecord<IN1>> inputIter1;
+ MutableObjectIterator<StreamRecord<IN2>> inputIter2;
+
+ private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
+ private CoInvokable<IN1, IN2, OUT> userFunction;
+ private int[] numberOfOutputChannels;
+ private static int numTasks;
+
+ public CoStreamTask() {
+
+ outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+ userFunction = null;
+ numTasks = newComponent();
+ instanceID = numTasks;
+ }
+
+ protected void setSerializers() {
+ byte[] operatorBytes = configuration.getBytes("operator", null);
+ String operatorName = configuration.getString("operatorName", "");
+
+ Object function = null;
+ try {
+ ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(operatorBytes));
+ function = in.readObject();
+
+ if (operatorName.equals("coMap")) {
+ setSerializer(function, CoMapFunction.class, 2);
+ setDeserializers(function, CoMapFunction.class);
+ } else {
+ throw new Exception("Wrong operator name!");
+ }
+
+ } catch (Exception e) {
+ throw new StreamComponentException(e);
+ }
+ }
+
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ private void setDeserializers(Object function, Class<? extends AbstractFunction> clazz) {
+
+ TupleTypeInfo<IN1> inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz,
+ function.getClass(), 0, null, null);
+ inTupleSerializer1 = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
+
+ inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
+ 1, null, null);
+ inTupleSerializer2 = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
+ }
+
+ @Override
+ public void registerInputOutput() {
+ initialize();
+
+ setSerializers();
+ setCollector();
+
+ // inputs1 = setConfigInputs();
+ setConfigInputs();
+
+ inputIter1 = createInputIterator(inputs1, inTupleSerializer1);
+
+ // inputs2 = setConfigInputs();
+ inputIter2 = createInputIterator(inputs2, inTupleSerializer2);
+
+ setConfigOutputs(outputs);
+
+ numberOfOutputChannels = new int[outputs.size()];
+ for (int i = 0; i < numberOfOutputChannels.length; i++) {
+ numberOfOutputChannels[i] = configuration.getInteger("channels_" + i, 0);
+ }
+
+ setInvokable();
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ @Override
+ protected void setInvokable() {
+ // Default value is a CoMapInvokable
+ Class<? extends CoInvokable> userFunctionClass = configuration.getClass("userfunction",
+ CoMapInvokable.class, CoInvokable.class);
+ userFunction = (CoInvokable<IN1, IN2, OUT>) getInvokable(userFunctionClass);
+ userFunction.initialize(collector, inputIter1, inTupleSerializer1, inputIter2,
+ inTupleSerializer2);
+ }
+
+ protected void setConfigInputs() throws StreamComponentException {
+ int numberOfInputs = configuration.getInteger("numberOfInputs", 0);
+
+ ArrayList<MutableRecordReader<IOReadableWritable>> inputList1 = new ArrayList<MutableRecordReader<IOReadableWritable>>();
+ ArrayList<MutableRecordReader<IOReadableWritable>> inputList2 = new ArrayList<MutableRecordReader<IOReadableWritable>>();
+
+ for (int i = 0; i < numberOfInputs; i++) {
+ int inputType = configuration.getInteger("inputType_" + i, 0);
+ switch (inputType) {
+ case 1:
+ inputList1.add(new MutableRecordReader<IOReadableWritable>(this));
+ break;
+ case 2:
+ inputList2.add(new MutableRecordReader<IOReadableWritable>(this));
+ break;
+ default:
+ throw new RuntimeException("Invalid input type number: " + inputType);
+ }
+ }
+
+ inputs1 = getInputs(inputList1);
+ inputs2 = getInputs(inputList2);
+ }
+
+ @SuppressWarnings("unchecked")
+ private MutableReader<IOReadableWritable> getInputs(
+ ArrayList<MutableRecordReader<IOReadableWritable>> inputList) {
+ if (inputList.size() == 1) {
+ return inputList.get(0);
+ } else if (inputList.size() > 1) {
+ return new MutableUnionRecordReader<IOReadableWritable>(
+ (MutableRecordReader<IOReadableWritable>[]) inputList.toArray());
+ }
+ return null;
+ }
+
+ @Override
+ public void invoke() throws Exception {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("TASK " + name + " invoked with instance id " + instanceID);
+ }
+
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
+ output.initializeSerializers();
+ }
+
+ userFunction.invoke();
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("TASK " + name + " invoke finished with instance id " + instanceID);
+ }
+
+ for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
+ output.flush();
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
new file mode 100644
index 0000000..b49620f
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/SingleInputAbstractStreamComponent.java
@@ -0,0 +1,129 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.streamcomponent;
+
+import java.io.ByteArrayInputStream;
+import java.io.ObjectInputStream;
+
+import org.apache.flink.api.common.functions.AbstractFunction;
+import org.apache.flink.api.java.functions.FilterFunction;
+import org.apache.flink.api.java.functions.FlatMapFunction;
+import org.apache.flink.api.java.functions.GroupReduceFunction;
+import org.apache.flink.api.java.functions.MapFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.runtime.io.network.api.MutableReader;
+import org.apache.flink.runtime.io.network.api.MutableRecordReader;
+import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.function.co.CoMapFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+
+public abstract class SingleInputAbstractStreamComponent<IN extends Tuple, OUT extends Tuple> extends
+ AbstractStreamComponent<OUT> {
+
+ protected StreamRecordSerializer<IN> inTupleSerializer = null;
+
+ protected void setSerializers() {
+ byte[] operatorBytes = configuration.getBytes("operator", null);
+ String operatorName = configuration.getString("operatorName", "");
+
+ Object function = null;
+ try {
+ ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(operatorBytes));
+ function = in.readObject();
+
+ if (operatorName.equals("flatMap")) {
+ setSerializerDeserializer(function, FlatMapFunction.class);
+ } else if (operatorName.equals("map")) {
+ setSerializerDeserializer(function, MapFunction.class);
+ } else if (operatorName.equals("batchReduce")) {
+ setSerializerDeserializer(function, GroupReduceFunction.class);
+ } else if (operatorName.equals("filter")) {
+ setDeserializer(function, FilterFunction.class);
+ setSerializer(function, FilterFunction.class, 0);
+ } else if (operatorName.equals("sink")) {
+ setDeserializer(function, SinkFunction.class);
+ } else if (operatorName.equals("source")) {
+ setSerializer(function, UserSourceInvokable.class, 0);
+ } else if (operatorName.equals("coMap")) {
+ setSerializer(function, CoMapFunction.class, 2);
+ //setDeserializers(function, CoMapFunction.class);
+ } else if (operatorName.equals("elements")) {
+ outTupleTypeInfo = new TupleTypeInfo<OUT>(TypeExtractor.getForObject(function));
+
+ outTupleSerializer = new StreamRecordSerializer<OUT>(outTupleTypeInfo.createSerializer());
+ outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(
+ outTupleSerializer);
+ } else {
+ throw new Exception("Wrong operator name: " + operatorName);
+ }
+
+ } catch (Exception e) {
+ throw new StreamComponentException(e);
+ }
+ }
+
+ private void setSerializerDeserializer(Object function, Class<? extends AbstractFunction> clazz) {
+ setDeserializer(function, clazz);
+ setSerializer(function, clazz, 1);
+ }
+
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ private void setDeserializer(Object function, Class<? extends AbstractFunction> clazz) {
+ TupleTypeInfo<IN> inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
+ 0, null, null);
+
+ inTupleSerializer = new StreamRecordSerializer(inTupleTypeInfo.createSerializer());
+ }
+
+ @SuppressWarnings("unchecked")
+ protected void setSinkSerializer() {
+ if (outSerializationDelegate != null) {
+ TupleTypeInfo<IN> inTupleTypeInfo = (TupleTypeInfo<IN>) outTupleTypeInfo;
+
+ inTupleSerializer = new StreamRecordSerializer<IN>(inTupleTypeInfo.createSerializer());
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ protected MutableReader<IOReadableWritable> getConfigInputs() throws StreamComponentException {
+ int numberOfInputs = configuration.getInteger("numberOfInputs", 0);
+
+ if (numberOfInputs < 2) {
+
+ return new MutableRecordReader<IOReadableWritable>(this);
+
+ } else {
+ MutableRecordReader<IOReadableWritable>[] recordReaders = (MutableRecordReader<IOReadableWritable>[]) new MutableRecordReader<?>[numberOfInputs];
+
+ for (int i = 0; i < numberOfInputs; i++) {
+ recordReaders[i] = new MutableRecordReader<IOReadableWritable>(this);
+ }
+ return new MutableUnionRecordReader<IOReadableWritable>(recordReaders);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
index b92e031..1b25285 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
@@ -30,7 +30,7 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.util.MutableObjectIterator;
import org.apache.flink.util.StringUtils;
-public class StreamIterationSink<IN extends Tuple> extends AbstractStreamComponent<IN, IN> {
+public class StreamIterationSink<IN extends Tuple> extends SingleInputAbstractStreamComponent<IN, IN> {
private static final Log LOG = LogFactory.getLog(StreamIterationSink.class);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
index 9f58842..f880470 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
@@ -32,7 +32,7 @@ import org.apache.flink.runtime.io.network.api.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-public class StreamIterationSource<OUT extends Tuple> extends AbstractStreamComponent<Tuple, OUT> {
+public class StreamIterationSource<OUT extends Tuple> extends SingleInputAbstractStreamComponent<Tuple, OUT> {
private static final Log LOG = LogFactory.getLog(StreamIterationSource.class);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
index 1433ebc..5e3457c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
@@ -28,7 +28,7 @@ import org.apache.flink.streaming.api.invokable.StreamRecordInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.util.MutableObjectIterator;
-public class StreamSink<IN extends Tuple> extends AbstractStreamComponent<IN, IN> {
+public class StreamSink<IN extends Tuple> extends SingleInputAbstractStreamComponent<IN, IN> {
private static final Log LOG = LogFactory.getLog(StreamSink.class);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
index 108916a..12c9ba3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
@@ -30,7 +30,7 @@ import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.invokable.UserSourceInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-public class StreamSource<OUT extends Tuple> extends AbstractStreamComponent<Tuple, OUT> {
+public class StreamSource<OUT extends Tuple> extends SingleInputAbstractStreamComponent<Tuple, OUT> {
private static final Log LOG = LogFactory.getLog(StreamSource.class);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
index 7da19aa..5032446 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
@@ -35,7 +35,7 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.util.MutableObjectIterator;
public class StreamTask<IN extends Tuple, OUT extends Tuple> extends
- AbstractStreamComponent<IN, OUT> {
+ SingleInputAbstractStreamComponent<IN, OUT> {
private static final Log LOG = LogFactory.getLog(StreamTask.class);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f436690f/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
new file mode 100644
index 0000000..00ae3b9
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoMapTest.java
@@ -0,0 +1,77 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.streaming.api.DataStream;
+import org.apache.flink.streaming.api.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.function.co.CoMapFunction;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CoMapTest implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private static Set<String> result = new HashSet<String>();
+ private static Set<String> expected = new HashSet<String>();
+
+ @Test
+ public void test() {
+ expected.add("a");
+ expected.add("b");
+ expected.add("c");
+ expected.add("1");
+ expected.add("2");
+ expected.add("3");
+ expected.add("4");
+
+ LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+ DataStream<Tuple1<Integer>> ds1 = env.fromElements(1, 2, 3, 4);
+
+ @SuppressWarnings("unused")
+ DataStream<Tuple1<Boolean>> ds2 = env.fromElements("a", "b", "c").coMapWith(new CoMapFunction<Tuple1<String>, Tuple1<Integer>, Tuple1<Boolean>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public Tuple1<Boolean> map1(Tuple1<String> value) {
+ System.out.println("1: " + value);
+ result.add(value.f0);
+ return new Tuple1<Boolean>(true);
+ }
+
+ @Override
+ public Tuple1<Boolean> map2(Tuple1<Integer> value) {
+ System.out.println("2: " +value);
+ result.add(value.f0.toString());
+ return new Tuple1<Boolean>(false);
+ }
+ }, ds1)
+ .print();
+
+ env.executeTest(32);
+ Assert.assertArrayEquals(expected.toArray(), result.toArray());
+ }
+}
[32/51] [abbrv] [streaming] Updated Streaming function interfaces to
match main project
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
index 2a8973c..ca9c1cb 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/FlatMapTest.java
@@ -25,7 +25,7 @@ import static org.junit.Assert.assertTrue;
import java.util.HashSet;
import java.util.Set;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
@@ -37,7 +37,7 @@ import org.junit.Test;
public class FlatMapTest {
- public static final class MyFlatMap extends RichFlatMapFunction<Integer, Integer> {
+ public static final class MyFlatMap implements FlatMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@@ -49,7 +49,7 @@ public class FlatMapTest {
}
- public static final class ParallelFlatMap extends RichFlatMapFunction<Integer, Integer> {
+ public static final class ParallelFlatMap implements FlatMapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -60,7 +60,7 @@ public class FlatMapTest {
}
- public static final class GenerateSequenceFlatMap extends RichFlatMapFunction<Long, Long> {
+ public static final class GenerateSequenceFlatMap implements FlatMapFunction<Long, Long> {
private static final long serialVersionUID = 1L;
@Override
@@ -71,7 +71,7 @@ public class FlatMapTest {
}
- public static final class MySink extends SinkFunction<Integer> {
+ public static final class MySink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -81,7 +81,7 @@ public class FlatMapTest {
}
- public static final class FromElementsSink extends SinkFunction<Integer> {
+ public static final class FromElementsSink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -91,7 +91,7 @@ public class FlatMapTest {
}
- public static final class FromCollectionSink extends SinkFunction<Integer> {
+ public static final class FromCollectionSink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -101,7 +101,7 @@ public class FlatMapTest {
}
- public static final class GenerateSequenceSink extends SinkFunction<Long> {
+ public static final class GenerateSequenceSink implements SinkFunction<Long> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
index 07478fb..73185df 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
@@ -27,7 +27,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
@@ -40,7 +40,7 @@ import org.junit.Test;
public class MapTest {
- public static final class MySource extends SourceFunction<Integer> {
+ public static final class MySource implements SourceFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -51,7 +51,7 @@ public class MapTest {
}
}
- public static final class MySource1 extends SourceFunction<Integer> {
+ public static final class MySource1 implements SourceFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -62,7 +62,7 @@ public class MapTest {
}
}
- public static final class MySource2 extends SourceFunction<Integer> {
+ public static final class MySource2 implements SourceFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -73,7 +73,7 @@ public class MapTest {
}
}
- public static final class MySource3 extends SourceFunction<Integer> {
+ public static final class MySource3 implements SourceFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -84,7 +84,7 @@ public class MapTest {
}
}
- public static final class MyMap extends RichMapFunction<Integer, Integer> {
+ public static final class MyMap implements MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -94,7 +94,7 @@ public class MapTest {
}
}
- public static final class MySingleJoinMap extends RichMapFunction<Integer, Integer> {
+ public static final class MySingleJoinMap implements MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -104,7 +104,7 @@ public class MapTest {
}
}
- public static final class MyMultipleJoinMap extends RichMapFunction<Integer, Integer> {
+ public static final class MyMultipleJoinMap implements MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -114,7 +114,7 @@ public class MapTest {
}
}
- public static final class MyFieldsMap extends RichMapFunction<Integer, Integer> {
+ public static final class MyFieldsMap implements MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
private int counter = 0;
@@ -128,7 +128,7 @@ public class MapTest {
}
}
- public static final class MyDiffFieldsMap extends RichMapFunction<Integer, Integer> {
+ public static final class MyDiffFieldsMap implements MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
private int counter = 0;
@@ -138,11 +138,11 @@ public class MapTest {
counter++;
if (counter > 3)
threeInAll = false;
- return value*value;
+ return value * value;
}
}
- public static final class MySink extends SinkFunction<Integer> {
+ public static final class MySink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -151,7 +151,7 @@ public class MapTest {
}
}
- public static final class MyBroadcastSink extends SinkFunction<Integer> {
+ public static final class MyBroadcastSink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -160,7 +160,7 @@ public class MapTest {
}
}
- public static final class MyShufflesSink extends SinkFunction<Integer> {
+ public static final class MyShufflesSink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -169,7 +169,7 @@ public class MapTest {
}
}
- public static final class MyFieldsSink extends SinkFunction<Integer> {
+ public static final class MyFieldsSink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -178,7 +178,7 @@ public class MapTest {
}
}
- public static final class MyDiffFieldsSink extends SinkFunction<Integer> {
+ public static final class MyDiffFieldsSink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -187,7 +187,7 @@ public class MapTest {
}
}
- public static final class MyGraphSink extends SinkFunction<Integer> {
+ public static final class MyGraphSink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
@@ -196,7 +196,7 @@ public class MapTest {
}
}
- public static final class JoinSink extends SinkFunction<Integer> {
+ public static final class JoinSink implements SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
index 566acd2..7d78d8f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
@@ -45,7 +45,7 @@ public class StreamComponentTest {
public static Map<Integer, Integer> data = new HashMap<Integer, Integer>();
- public static class MySource extends SourceFunction<Tuple1<Integer>> {
+ public static class MySource implements SourceFunction<Tuple1<Integer>> {
private static final long serialVersionUID = 1L;
private Tuple1<Integer> tuple = new Tuple1<Integer>(0);
@@ -82,7 +82,7 @@ public class StreamComponentTest {
// }
// }
- public static class MySink extends SinkFunction<Tuple2<Integer, Integer>> {
+ public static class MySink implements SinkFunction<Tuple2<Integer, Integer>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
index 04b8ee2..c5b51e6 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
@@ -19,7 +19,7 @@
package org.apache.flink.streaming.examples.basictopology;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
@@ -28,7 +28,7 @@ import org.apache.flink.util.Collector;
public class BasicTopology {
- public static class BasicSource extends SourceFunction<Tuple1<String>> {
+ public static class BasicSource implements SourceFunction<Tuple1<String>> {
private static final long serialVersionUID = 1L;
Tuple1<String> tuple = new Tuple1<String>("streaming");
@@ -42,7 +42,7 @@ public class BasicTopology {
}
}
- public static class BasicMap extends RichMapFunction<Tuple1<String>, Tuple1<String>> {
+ public static class BasicMap implements MapFunction<Tuple1<String>, Tuple1<String>> {
private static final long serialVersionUID = 1L;
// map to the same tuple
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
index 5704723..bb5cc07 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
@@ -37,7 +37,7 @@ public class CellInfoLocal {
private final static int PARALLELISM = 1;
private final static int SOURCE_PARALLELISM = 1;
- private final static class QuerySource extends
+ private final static class QuerySource implements
SourceFunction<Tuple4<Boolean, Integer, Long, Integer>> {
private static final long serialVersionUID = 1L;
@@ -56,7 +56,7 @@ public class CellInfoLocal {
}
}
- public final static class InfoSource extends
+ public final static class InfoSource implements
SourceFunction<Tuple4<Boolean, Integer, Long, Integer>> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
index 8a76d0a..d2162fa 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
@@ -22,7 +22,7 @@ package org.apache.flink.streaming.examples.iterative.collaborativefilter;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple4;
-public class CollaborativeFilteringSink extends
+public class CollaborativeFilteringSink implements
SinkFunction<Tuple4<Integer, Integer, Integer, Long>> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
index 4776b20..6e57739 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
@@ -26,7 +26,7 @@ import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.util.Collector;
-public class CollaborativeFilteringSource extends
+public class CollaborativeFilteringSource implements
SourceFunction<Tuple4<Integer, Integer, Integer, Long>> {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java
index 047e7bf..c549d13 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSink.java
@@ -22,7 +22,7 @@ package org.apache.flink.streaming.examples.iterative.kmeans;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple3;
-public class KMeansSink extends SinkFunction<Tuple3<Integer, Integer, Long>> {
+public class KMeansSink implements SinkFunction<Tuple3<Integer, Integer, Long>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java
index 86720a2..ea50231 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/kmeans/KMeansSource.java
@@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.util.Collector;
-public class KMeansSource extends SourceFunction<Tuple2<String, Long>> {
+public class KMeansSource implements SourceFunction<Tuple2<String, Long>> {
private static final long serialVersionUID = 1L;
private final long DEFAULT_SEED = 4650285087650871364L;
private Random random = new Random(DEFAULT_SEED);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java
index 9b0e23a..77f5c10 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSink.java
@@ -22,7 +22,7 @@ package org.apache.flink.streaming.examples.iterative.pagerank;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple3;
-public class PageRankSink extends SinkFunction<Tuple3<Integer, Float, Long>> {
+public class PageRankSink implements SinkFunction<Tuple3<Integer, Float, Long>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
index 3269856..6b38f2f 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/pagerank/PageRankSource.java
@@ -26,7 +26,7 @@ import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.util.Collector;
-public class PageRankSource extends SourceFunction<Tuple3<Integer, Integer, Long>> {
+public class PageRankSource implements SourceFunction<Tuple3<Integer, Integer, Long>> {
private static final long serialVersionUID = 1L;
private Tuple3<Integer, Integer, Long> outRecord = new Tuple3<Integer, Integer, Long>();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java
index 0b10be0..ec84bd1 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSink.java
@@ -22,7 +22,7 @@ package org.apache.flink.streaming.examples.iterative.sssp;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple3;
-public class SSSPSink extends SinkFunction<Tuple3<Integer, Integer, Long>> {
+public class SSSPSink implements SinkFunction<Tuple3<Integer, Integer, Long>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
index 15be203..7936efc 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iterative/sssp/SSSPSource.java
@@ -26,7 +26,7 @@ import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.util.Collector;
-public class SSSPSource extends SourceFunction<Tuple3<Integer, Integer, Long>> {
+public class SSSPSource implements SourceFunction<Tuple3<Integer, Integer, Long>> {
private static final long serialVersionUID = 1L;
private Tuple3<Integer, Integer, Long> outRecord = new Tuple3<Integer, Integer, Long>();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java
index 1d87a20..4cf7ab6 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSink.java
@@ -22,7 +22,7 @@ package org.apache.flink.streaming.examples.join;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.api.java.tuple.Tuple3;
-public class JoinSink extends SinkFunction<Tuple3<String, Integer, Integer>> {
+public class JoinSink implements SinkFunction<Tuple3<String, Integer, Integer>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java
index d178033..5d59ee9 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceOne.java
@@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.util.Collector;
-public class JoinSourceOne extends SourceFunction<Tuple3<String, String, Integer>> {
+public class JoinSourceOne implements SourceFunction<Tuple3<String, String, Integer>> {
private static final long serialVersionUID = 6670933703432267728L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java
index 3730534..032db06 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinSourceTwo.java
@@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.util.Collector;
-public class JoinSourceTwo extends SourceFunction<Tuple3<String, String, Integer>> {
+public class JoinSourceTwo implements SourceFunction<Tuple3<String, String, Integer>> {
private static final long serialVersionUID = -5897483980082089771L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index 05a7eb1..6264cb9 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -29,7 +29,7 @@ import org.apache.flink.util.Collector;
public class IncrementalLearningSkeleton {
// Source for feeding new data for prediction
- public static class NewDataSource extends SourceFunction<Tuple1<Integer>> {
+ public static class NewDataSource implements SourceFunction<Tuple1<Integer>> {
private static final long serialVersionUID = 1L;
@Override
@@ -46,7 +46,7 @@ public class IncrementalLearningSkeleton {
}
// Source for feeding new training data for partial model building
- public static class TrainingDataSource extends SourceFunction<Tuple1<Integer>> {
+ public static class TrainingDataSource implements SourceFunction<Tuple1<Integer>> {
private static final long serialVersionUID = 1L;
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
index 1b1cfe1..366e7b5 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalOLS.java
@@ -34,7 +34,7 @@ import org.apache.flink.util.Collector;
public class IncrementalOLS {
- public static class NewDataSource extends SourceFunction<Tuple2<Boolean, Double[]>> {
+ public static class NewDataSource implements SourceFunction<Tuple2<Boolean, Double[]>> {
private static final long serialVersionUID = 1L;
Random rnd = new Random();
@@ -55,7 +55,7 @@ public class IncrementalOLS {
}
}
- public static class TrainingDataSource extends SourceFunction<Tuple2<Double, Double[]>> {
+ public static class TrainingDataSource implements SourceFunction<Tuple2<Double, Double[]>> {
private static final long serialVersionUID = 1L;
Random rnd = new Random();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java
index 9e16ef8..eacd790 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceOne.java
@@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.util.Collector;
-public class WindowJoinSourceOne extends SourceFunction<Tuple4<String, String, Integer, Long>> {
+public class WindowJoinSourceOne implements SourceFunction<Tuple4<String, String, Integer, Long>> {
private static final long serialVersionUID = 6670933703432267728L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java
index dfac58d..c4441c0 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinSourceTwo.java
@@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.util.Collector;
-public class WindowJoinSourceTwo extends SourceFunction<Tuple4<String, String, Integer, Long>> {
+public class WindowJoinSourceTwo implements SourceFunction<Tuple4<String, String, Integer, Long>> {
private static final long serialVersionUID = -5897483980082089771L;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
index 7142b58..f106afe 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
@@ -22,10 +22,10 @@ package org.apache.flink.streaming.examples.wordcount;
import java.util.HashMap;
import java.util.Map;
-import org.apache.flink.api.java.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.common.functions.MapFunction;
-public class WordCountCounter extends RichMapFunction<String, Tuple2<String, Integer>> {
+public class WordCountCounter implements MapFunction<String, Tuple2<String, Integer>> {
private static final long serialVersionUID = 1L;
private Map<String, Integer> wordCounts = new HashMap<String, Integer>();
@@ -33,7 +33,7 @@ public class WordCountCounter extends RichMapFunction<String, Tuple2<String, Int
private Integer count = 0;
private Tuple2<String, Integer> outTuple = new Tuple2<String, Integer>();
-
+
// Increments the counter of the occurrence of the input word
@Override
public Tuple2<String, Integer> map(String inTuple) throws Exception {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
index 6295d02..eb0dfd3 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
@@ -19,16 +19,16 @@
package org.apache.flink.streaming.examples.wordcount;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.util.Collector;
-public class WordCountSplitter extends RichFlatMapFunction<String, String> {
+public class WordCountSplitter implements FlatMapFunction<String, String> {
private static final long serialVersionUID = 1L;
// Splits the lines according on spaces
@Override
public void flatMap(String inTuple, Collector<String> out) throws Exception {
-
+
for (String word : inTuple.split(" ")) {
out.collect(word);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
new file mode 100644
index 0000000..2c386b7
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
@@ -0,0 +1 @@
+f1b947a26b33b32f1de2cdd841f7b4c8
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
new file mode 100644
index 0000000..6499b43
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
@@ -0,0 +1 @@
+6e47046882bad158b0efbb84cd5cb987
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
new file mode 100644
index 0000000..6526a51
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
@@ -0,0 +1 @@
+4bb8c10cdde12a4953250423266465cc
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e73ea295/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
new file mode 100644
index 0000000..365f210
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
@@ -0,0 +1 @@
+7002e15fe547614160a0df6f22a5b8d0
[27/51] [abbrv] [streaming] Added support for simple types instead of
Tuple1 in the API
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
index 2cea800..2a35de5 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/MapTest.java
@@ -27,181 +27,180 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
+import org.apache.flink.api.java.functions.MapFunction;
import org.apache.flink.streaming.api.DataStream;
import org.apache.flink.streaming.api.LocalStreamEnvironment;
import org.apache.flink.streaming.api.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.LogUtils;
-import org.junit.Test;
-import org.apache.flink.api.java.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
import org.apache.log4j.Level;
+import org.junit.Test;
public class MapTest {
- public static final class MySource extends SourceFunction<Tuple1<Integer>> {
+ public static final class MySource extends SourceFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Collector<Tuple1<Integer>> collector) throws Exception {
+ public void invoke(Collector<Integer> collector) throws Exception {
for (int i = 0; i < 10; i++) {
- collector.collect(new Tuple1<Integer>(i));
+ collector.collect(i);
}
}
}
-
- public static final class MySource1 extends SourceFunction<Tuple1<Integer>> {
+
+ public static final class MySource1 extends SourceFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Collector<Tuple1<Integer>> collector) throws Exception {
+ public void invoke(Collector<Integer> collector) throws Exception {
for (int i = 0; i < 5; i++) {
- collector.collect(new Tuple1<Integer>(i));
+ collector.collect(i);
}
}
}
-
- public static final class MySource2 extends SourceFunction<Tuple1<Integer>> {
+
+ public static final class MySource2 extends SourceFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Collector<Tuple1<Integer>> collector) throws Exception {
+ public void invoke(Collector<Integer> collector) throws Exception {
for (int i = 5; i < 10; i++) {
- collector.collect(new Tuple1<Integer>(i));
+ collector.collect(i);
}
}
}
-
- public static final class MySource3 extends SourceFunction<Tuple1<Integer>> {
+
+ public static final class MySource3 extends SourceFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Collector<Tuple1<Integer>> collector) throws Exception {
+ public void invoke(Collector<Integer> collector) throws Exception {
for (int i = 10; i < 15; i++) {
- collector.collect(new Tuple1<Integer>(i));
+ collector.collect(new Integer(i));
}
}
}
- public static final class MyMap extends MapFunction<Tuple1<Integer>, Tuple1<Integer>> {
+ public static final class MyMap extends MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
- public Tuple1<Integer> map(Tuple1<Integer> value) throws Exception {
+ public Integer map(Integer value) throws Exception {
map++;
- return new Tuple1<Integer>(value.f0 * value.f0);
+ return value * value;
}
}
-
- public static final class MySingleJoinMap extends MapFunction<Tuple1<Integer>, Tuple1<Integer>> {
+
+ public static final class MySingleJoinMap extends MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
- public Tuple1<Integer> map(Tuple1<Integer> value) throws Exception {
- singleJoinSetResult.add(value.f0);
- return new Tuple1<Integer>(value.f0);
+ public Integer map(Integer value) throws Exception {
+ singleJoinSetResult.add(value);
+ return value;
}
}
-
- public static final class MyMultipleJoinMap extends MapFunction<Tuple1<Integer>, Tuple1<Integer>> {
+
+ public static final class MyMultipleJoinMap extends MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
- public Tuple1<Integer> map(Tuple1<Integer> value) throws Exception {
- multipleJoinSetResult.add(value.f0);
- return new Tuple1<Integer>(value.f0);
+ public Integer map(Integer value) throws Exception {
+ multipleJoinSetResult.add(value);
+ return value;
}
}
- public static final class MyFieldsMap extends MapFunction<Tuple1<Integer>, Tuple1<Integer>> {
+ public static final class MyFieldsMap extends MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
private int counter = 0;
@Override
- public Tuple1<Integer> map(Tuple1<Integer> value) throws Exception {
+ public Integer map(Integer value) throws Exception {
counter++;
if (counter == MAXSOURCE)
allInOne = true;
- return new Tuple1<Integer>(value.f0 * value.f0);
+ return value * value;
}
}
-
- public static final class MyDiffFieldsMap extends MapFunction<Tuple1<Integer>, Tuple1<Integer>> {
+
+ public static final class MyDiffFieldsMap extends MapFunction<Integer, Integer> {
private static final long serialVersionUID = 1L;
private int counter = 0;
@Override
- public Tuple1<Integer> map(Tuple1<Integer> value) throws Exception {
+ public Integer map(Integer value) throws Exception {
counter++;
if (counter > 3)
threeInAll = false;
- return new Tuple1<Integer>(value.f0 * value.f0);
+ return value*value;
}
}
- public static final class MySink extends SinkFunction<Tuple1<Integer>> {
+ public static final class MySink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
- result.add(tuple.f0);
+ public void invoke(Integer tuple) {
+ result.add(tuple);
}
}
- public static final class MyBroadcastSink extends SinkFunction<Tuple1<Integer>> {
+ public static final class MyBroadcastSink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
+ public void invoke(Integer tuple) {
broadcastResult++;
}
}
- public static final class MyShufflesSink extends SinkFunction<Tuple1<Integer>> {
+ public static final class MyShufflesSink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
+ public void invoke(Integer tuple) {
shuffleResult++;
}
}
- public static final class MyFieldsSink extends SinkFunction<Tuple1<Integer>> {
+ public static final class MyFieldsSink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
+ public void invoke(Integer tuple) {
fieldsResult++;
}
}
-
- public static final class MyDiffFieldsSink extends SinkFunction<Tuple1<Integer>> {
+
+ public static final class MyDiffFieldsSink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
+ public void invoke(Integer tuple) {
diffFieldsResult++;
}
}
-
- public static final class MyGraphSink extends SinkFunction<Tuple1<Integer>> {
+
+ public static final class MyGraphSink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
+ public void invoke(Integer tuple) {
graphResult++;
}
}
-
- public static final class JoinSink extends SinkFunction<Tuple1<Integer>> {
+
+ public static final class JoinSink extends SinkFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
- public void invoke(Tuple1<Integer> tuple) {
+ public void invoke(Integer tuple) {
}
}
@@ -210,14 +209,14 @@ public class MapTest {
private static int broadcastResult = 0;
private static int shuffleResult = 0;
@SuppressWarnings("unused")
- private static int fieldsResult = 0;
+ private static int fieldsResult = 0;
private static int diffFieldsResult = 0;
@SuppressWarnings("unused")
- private static int graphResult = 0;
+ private static int graphResult = 0;
@SuppressWarnings("unused")
- private static int map = 0;
+ private static int map = 0;
@SuppressWarnings("unused")
- private static final int PARALLELISM = 1;
+ private static final int PARALLELISM = 1;
private static final long MEMORYSIZE = 32;
private static final int MAXSOURCE = 10;
private static boolean allInOne = false;
@@ -235,139 +234,110 @@ public class MapTest {
expected.add(i * i);
}
}
-
+
private static void fillFromCollectionSet() {
- if(fromCollectionSet.isEmpty()){
+ if (fromCollectionSet.isEmpty()) {
for (int i = 0; i < 10; i++) {
fromCollectionSet.add(i);
}
}
}
-
+
private static void fillFromCollectionFieldsSet() {
- if(fromCollectionFields.isEmpty()){
+ if (fromCollectionFields.isEmpty()) {
for (int i = 0; i < MAXSOURCE; i++) {
-
+
fromCollectionFields.add(5);
}
}
}
-
+
private static void fillFromCollectionDiffFieldsSet() {
- if(fromCollectionDiffFieldsSet.isEmpty()){
+ 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 {
LogUtils.initializeDefaultConsoleLogger(Level.OFF, Level.OFF);
- //mapTest
+ // mapTest
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(3);
fillFromCollectionSet();
-
- @SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream = env.fromCollection(fromCollectionSet)
- .map(new MyMap()).addSink(new MySink());
+ @SuppressWarnings("unused")
+ DataStream<Integer> dataStream = env.fromCollection(fromCollectionSet).map(new MyMap())
+ .addSink(new MySink());
fillExpectedList();
-
-
- //broadcastSinkTest
+
+ // broadcastSinkTest
fillFromCollectionSet();
-
+
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream1 = env
- .fromCollection(fromCollectionSet)
- .broadcast()
- .map(new MyMap())
- .addSink(new MyBroadcastSink());
-
-
- //shuffleSinkTest
+ DataStream<Integer> dataStream1 = env.fromCollection(fromCollectionSet).broadcast()
+ .map(new MyMap()).addSink(new MyBroadcastSink());
+
+ // shuffleSinkTest
fillFromCollectionSet();
-
+
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream2 = env
- .fromCollection(fromCollectionSet)
- .map(new MyMap()).setParallelism(3)
- .addSink(new MyShufflesSink());
+ DataStream<Integer> dataStream2 = env.fromCollection(fromCollectionSet).map(new MyMap())
+ .setParallelism(3).addSink(new MyShufflesSink());
-
- //fieldsMapTest
+ // fieldsMapTest
fillFromCollectionFieldsSet();
-
+
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream3 = env
- .fromCollection(fromCollectionFields)
- .partitionBy(0)
- .map(new MyFieldsMap())
- .addSink(new MyFieldsSink());
-
-
- //diffFieldsMapTest
+ DataStream<Integer> dataStream3 = env.fromCollection(fromCollectionFields).partitionBy(0)
+ .map(new MyFieldsMap()).addSink(new MyFieldsSink());
+
+ // diffFieldsMapTest
fillFromCollectionDiffFieldsSet();
-
+
@SuppressWarnings("unused")
- DataStream<Tuple1<Integer>> dataStream4 = env
- .fromCollection(fromCollectionDiffFieldsSet)
- .partitionBy(0)
- .map(new MyDiffFieldsMap())
- .addSink(new MyDiffFieldsSink());
-
-
- //singleConnectWithTest
- DataStream<Tuple1<Integer>> source1 = env.addSource(new MySource1(),
- 1);
-
+ DataStream<Integer> dataStream4 = env.fromCollection(fromCollectionDiffFieldsSet)
+ .partitionBy(0).map(new MyDiffFieldsMap()).addSink(new MyDiffFieldsSink());
+
+ // singleConnectWithTest
+ DataStream<Integer> source1 = env.addSource(new MySource1(), 1);
+
@SuppressWarnings({ "unused", "unchecked" })
- DataStream<Tuple1<Integer>> source2 = env
- .addSource(new MySource2(), 1)
- .connectWith(source1)
- .partitionBy(0)
- .map(new MySingleJoinMap()).setParallelism(1)
+ DataStream<Integer> source2 = env.addSource(new MySource2(), 1).connectWith(source1)
+ .partitionBy(0).map(new MySingleJoinMap()).setParallelism(1)
.addSink(new JoinSink());
-
fillSingleJoinSet();
-
-
- //multipleConnectWithTest
- DataStream<Tuple1<Integer>> source3 = env.addSource(new MySource1(),
- 1);
-
- DataStream<Tuple1<Integer>> source4 = env.addSource(new MySource2(),
- 1);
-
+
+ // multipleConnectWithTest
+ DataStream<Integer> source3 = env.addSource(new MySource1(), 1);
+
+ DataStream<Integer> source4 = env.addSource(new MySource2(), 1);
+
@SuppressWarnings({ "unused", "unchecked" })
- DataStream<Tuple1<Integer>> source5 = env
- .addSource(new MySource3(), 1)
- .connectWith(source3, source4)
- .partitionBy(0)
- .map(new MyMultipleJoinMap()).setParallelism(1)
- .addSink(new JoinSink());
+ DataStream<Integer> source5 = env.addSource(new MySource3(), 1)
+ .connectWith(source3, source4).partitionBy(0).map(new MyMultipleJoinMap())
+ .setParallelism(1).addSink(new JoinSink());
+
+ env.executeTest(MEMORYSIZE);
- env.executeTest(MEMORYSIZE);
-
fillMultipleJoinSet();
-
+
assertTrue(expected.equals(result));
assertEquals(30, broadcastResult);
assertEquals(10, shuffleResult);
@@ -376,7 +346,7 @@ public class MapTest {
assertEquals(9, diffFieldsResult);
assertEquals(singleJoinSetExpected, singleJoinSetResult);
assertEquals(multipleJoinSetExpected, multipleJoinSetResult);
-
+
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
index e99518c..463abc9 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
@@ -42,6 +42,6 @@ public class MockRecordWriter extends RecordWriter<SerializationDelegate<StreamR
@Override
public void emit(SerializationDelegate<StreamRecord<Tuple1<Integer>>> record) {
- emittedRecords.add(record.getInstance().getTuple().f0);
+ emittedRecords.add(record.getInstance().getObject().f0);
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java
index 716a869..01c2092 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/partitioner/FieldsPartitionerTest.java
@@ -33,9 +33,9 @@ public class FieldsPartitionerTest {
private FieldsPartitioner<Tuple> fieldsPartitioner;
private StreamRecord<Tuple> streamRecord1 = new StreamRecord<Tuple>()
- .setTuple(new Tuple2<String, Integer>("test", 0));
+ .setObject(new Tuple2<String, Integer>("test", 0));
private StreamRecord<Tuple> streamRecord2 = new StreamRecord<Tuple>()
- .setTuple(new Tuple2<String, Integer>("test", 42));
+ .setObject(new Tuple2<String, Integer>("test", 42));
private SerializationDelegate<StreamRecord<Tuple>> sd1 = new SerializationDelegate<StreamRecord<Tuple>>(
null);
private SerializationDelegate<StreamRecord<Tuple>> sd2 = new SerializationDelegate<StreamRecord<Tuple>>(
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
index dceaf46..4ca191e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
@@ -21,6 +21,8 @@ package org.apache.flink.streaming.util.serialization;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import org.apache.commons.lang3.SerializationUtils;
import org.apache.flink.api.java.functions.MapFunction;
@@ -50,15 +52,20 @@ public class TypeSerializationTest {
TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>> ser2 = (TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>>) SerializationUtils
.deserialize(serializedType);
- assertNotNull(ser.getInputTupleTypeInfo1());
- assertNotNull(ser2.getInputTupleTypeInfo1());
-
- assertNotNull(ser.getOutputTupleTypeInfo());
- assertNotNull(ser2.getOutputTupleTypeInfo());
-
- assertEquals(ser.getInputTupleTypeInfo1(), ser2.getInputTupleTypeInfo1());
- assertEquals(ser.getInputTupleTypeInfo2(), ser2.getInputTupleTypeInfo2());
- assertEquals(ser.getOutputTupleTypeInfo(), ser2.getOutputTupleTypeInfo());
+ assertNotNull(ser.getInputTypeInfo1());
+ assertNotNull(ser2.getInputTypeInfo1());
+
+ assertNotNull(ser.getOutputTypeInfo());
+ assertNotNull(ser2.getOutputTypeInfo());
+
+ assertEquals(ser.getInputTypeInfo1(), ser2.getInputTypeInfo1());
+ try {
+ ser.getInputTypeInfo2();
+ fail();
+ } catch (RuntimeException e) {
+ assertTrue(true);
+ }
+ assertEquals(ser.getOutputTypeInfo(), ser2.getOutputTypeInfo());
}
@SuppressWarnings("unchecked")
@@ -68,24 +75,29 @@ public class TypeSerializationTest {
Integer instance2 = null;
Integer instance3 = new Integer(34);
- TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>> ser = new ObjectTypeWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>>(
+ TypeSerializerWrapper<Integer, Integer, Integer> ser = new ObjectTypeWrapper<Integer, Integer, Integer>(
instance1, instance2, instance3);
- System.out.println(ser.getInputTupleTypeInfo1());
+ // System.out.println(ser.getInputTupleTypeInfo1());
byte[] serializedType = SerializationUtils.serialize(ser);
TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>> ser2 = (TypeSerializerWrapper<Tuple1<Integer>, Tuple, Tuple1<Integer>>) SerializationUtils
.deserialize(serializedType);
- assertNotNull(ser.getInputTupleTypeInfo1());
- assertNotNull(ser2.getInputTupleTypeInfo1());
+ assertNotNull(ser.getInputTypeInfo1());
+ assertNotNull(ser2.getInputTypeInfo1());
- assertNotNull(ser.getOutputTupleTypeInfo());
- assertNotNull(ser2.getOutputTupleTypeInfo());
-
- assertEquals(ser.getInputTupleTypeInfo1(), ser2.getInputTupleTypeInfo1());
- assertEquals(ser.getInputTupleTypeInfo2(), ser2.getInputTupleTypeInfo2());
- assertEquals(ser.getOutputTupleTypeInfo(), ser2.getOutputTupleTypeInfo());
+ assertNotNull(ser.getOutputTypeInfo());
+ assertNotNull(ser2.getOutputTypeInfo());
+
+ assertEquals(ser.getInputTypeInfo1(), ser2.getInputTypeInfo1());
+ try {
+ ser.getInputTypeInfo2();
+ fail();
+ } catch (RuntimeException e) {
+ assertTrue(true);
+ }
+ assertEquals(ser.getOutputTypeInfo(), ser2.getOutputTypeInfo());
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
index 42c0115..53c23d6 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountCounter.java
@@ -23,10 +23,9 @@ import java.util.HashMap;
import java.util.Map;
import org.apache.flink.api.java.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
-public class WordCountCounter extends MapFunction<Tuple1<String>, Tuple2<String, Integer>> {
+public class WordCountCounter extends MapFunction<String, Tuple2<String, Integer>> {
private static final long serialVersionUID = 1L;
private Map<String, Integer> wordCounts = new HashMap<String, Integer>();
@@ -37,8 +36,8 @@ public class WordCountCounter extends MapFunction<Tuple1<String>, Tuple2<String,
// Increments the counter of the occurrence of the input word
@Override
- public Tuple2<String, Integer> map(Tuple1<String> inTuple) throws Exception {
- word = inTuple.f0;
+ public Tuple2<String, Integer> map(String inTuple) throws Exception {
+ word = inTuple;
if (wordCounts.containsKey(word)) {
count = wordCounts.get(word) + 1;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
index 7d13a5c..f77ab37 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
@@ -32,7 +32,7 @@ public class WordCountLocal {
public static void main(String[] args) {
TestDataUtil.downloadIfNotExists("hamlet.txt");
- StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
DataStream<Tuple2<String, Integer>> dataStream = env
.readTextFile("src/test/resources/testdata/hamlet.txt")
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1162caca/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
index afe25ce..f3e9310 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountSplitter.java
@@ -20,21 +20,17 @@
package org.apache.flink.streaming.examples.wordcount;
import org.apache.flink.api.java.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.util.Collector;
-public class WordCountSplitter extends FlatMapFunction<Tuple1<String>, Tuple1<String>> {
+public class WordCountSplitter extends FlatMapFunction<String, String> {
private static final long serialVersionUID = 1L;
- private Tuple1<String> outTuple = new Tuple1<String>();
-
// Splits the lines according on spaces
@Override
- public void flatMap(Tuple1<String> inTuple, Collector<Tuple1<String>> out) throws Exception {
+ public void flatMap(String inTuple, Collector<String> out) throws Exception {
- for (String word : inTuple.f0.split(" ")) {
- outTuple.f0 = word;
- out.collect(outTuple);
+ for (String word : inTuple.split(" ")) {
+ out.collect(word);
}
}
}
\ No newline at end of file