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]);
+	}
+}