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:51 UTC
[14/51] [abbrv] git commit: [streaming] Distributed partitioner added
[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]);
+ }
+}