You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/10/06 23:01:00 UTC

[jira] [Commented] (FLINK-8532) RebalancePartitioner should use Random value for its first partition

    [ https://issues.apache.org/jira/browse/FLINK-8532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16640899#comment-16640899 ] 

ASF GitHub Bot commented on FLINK-8532:
---------------------------------------

asfgit closed pull request #6544: [FLINK-8532] [Streaming] modify RebalancePartitioner to use a random partition as its first partition
URL: https://github.com/apache/flink/pull/6544
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
index bb88d17cf9c..2c08884b829 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
@@ -21,6 +21,8 @@
 import org.apache.flink.runtime.plugable.SerializationDelegate;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
+import java.util.concurrent.ThreadLocalRandom;
+
 /**
  * Partitioner that distributes the data equally by cycling through the output
  * channels.
@@ -31,18 +33,27 @@
 public class RebalancePartitioner<T> extends StreamPartitioner<T> {
 	private static final long serialVersionUID = 1L;
 
-	private final int[] returnArray = new int[] {-1};
+	private final int[] returnArray = new int[] {Integer.MAX_VALUE - 1};
 
 	@Override
 	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
 			int numberOfOutputChannels) {
 		int newChannel = ++this.returnArray[0];
 		if (newChannel >= numberOfOutputChannels) {
-			this.returnArray[0] = 0;
+			this.returnArray[0] = resetValue(record, numberOfOutputChannels, newChannel);
 		}
 		return this.returnArray;
 	}
 
+	private int resetValue(SerializationDelegate<StreamRecord<T>> record,
+			int numberOfOutputChannels, int newChannel) {
+		if (newChannel == Integer.MAX_VALUE) {
+			// Initializes the first partition, this branch is only entered when initializing.
+			return ThreadLocalRandom.current().nextInt(numberOfOutputChannels);
+		}
+		return 0;
+	}
+
 	public StreamPartitioner<T> copy() {
 		return this;
 	}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java
index 85410f3de98..0cf022b3f27 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java
@@ -25,6 +25,7 @@
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests for {@link RebalancePartitioner}.
@@ -52,9 +53,11 @@ public void testSelectChannelsLength() {
 	@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]);
+		int initialChannel = distributePartitioner.selectChannels(sd, 3)[0];
+		assertTrue(0 <= initialChannel);
+		assertTrue(3 > initialChannel);
+		assertEquals((initialChannel + 1) % 3, distributePartitioner.selectChannels(sd, 3)[0]);
+		assertEquals((initialChannel + 2) % 3, distributePartitioner.selectChannels(sd, 3)[0]);
+		assertEquals((initialChannel + 3) % 3, distributePartitioner.selectChannels(sd, 3)[0]);
 	}
 }
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java
index 23fc2eb3df1..cdea26d6552 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java
@@ -219,18 +219,42 @@ private static void verifyBroadcastPartitioning(List<Tuple2<Integer, String>> br
 	}
 
 	private static void verifyRebalancePartitioning(List<Tuple2<Integer, String>> rebalancePartitionResult) {
-		List<Tuple2<Integer, String>> expected = Arrays.asList(
+		List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+			Arrays.asList(
 				new Tuple2<Integer, String>(0, "a"),
 				new Tuple2<Integer, String>(1, "b"),
 				new Tuple2<Integer, String>(2, "b"),
 				new Tuple2<Integer, String>(0, "a"),
 				new Tuple2<Integer, String>(1, "a"),
 				new Tuple2<Integer, String>(2, "c"),
-				new Tuple2<Integer, String>(0, "a"));
+				new Tuple2<Integer, String>(0, "a")),
 
-		assertEquals(
-				new HashSet<Tuple2<Integer, String>>(expected),
-				new HashSet<Tuple2<Integer, String>>(rebalancePartitionResult));
+			Arrays.asList(
+				new Tuple2<Integer, String>(1, "a"),
+				new Tuple2<Integer, String>(2, "b"),
+				new Tuple2<Integer, String>(0, "b"),
+				new Tuple2<Integer, String>(1, "a"),
+				new Tuple2<Integer, String>(2, "a"),
+				new Tuple2<Integer, String>(0, "c"),
+				new Tuple2<Integer, String>(1, "a")),
+
+			Arrays.asList(
+				new Tuple2<Integer, String>(2, "a"),
+				new Tuple2<Integer, String>(0, "b"),
+				new Tuple2<Integer, String>(1, "b"),
+				new Tuple2<Integer, String>(2, "a"),
+				new Tuple2<Integer, String>(0, "a"),
+				new Tuple2<Integer, String>(1, "c"),
+				new Tuple2<Integer, String>(2, "a")));
+
+		int matchedNum = 0;
+		for (List<Tuple2<Integer, String>> e : expected) {
+			if (new HashSet<Tuple2<Integer, String>>(e).equals(new HashSet<Tuple2<Integer, String>>(rebalancePartitionResult))) {
+				++matchedNum;
+			}
+		}
+
+		assertEquals(1, matchedNum);
 	}
 
 	private static void verifyGlobalPartitioning(List<Tuple2<Integer, String>> globalPartitionResult) {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> RebalancePartitioner should use Random value for its first partition
> --------------------------------------------------------------------
>
>                 Key: FLINK-8532
>                 URL: https://issues.apache.org/jira/browse/FLINK-8532
>             Project: Flink
>          Issue Type: Improvement
>          Components: DataStream API
>            Reporter: Yuta Morisawa
>            Assignee: Guibo Pan
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.7.0
>
>
> In some conditions, RebalancePartitioner doesn't balance data correctly because it use the same value for selecting next operators.
> RebalancePartitioner initializes its partition id using the same value in every threads, so it indeed balances data, but at one moment the amount of data in each operator is skew.
> Particularly, when the data rate of  former operators is equal , data skew becomes severe.
>  
>  
> Example:
> Consider a simple operator chain.
> -> map1 -> rebalance -> map2 ->
> Each map operator(map1, map2) contains three subtasks(subtask 1, 2, 3, 4, 5, 6).
> map1          map2
>  st1              st4
>  st2              st5
>  st3              st6
>  
> At the beginning, every subtasks in map1 sends data to st4 in map2 because they use the same initial parition id.
> Next time the map1 receive data st1,2,3 send data to st5 because they increment its partition id when they processed former data.
> In my environment,  it takes twice the time to process data when I use RebalancePartitioner  as long as I use other partitioners(rescale, keyby).
>  
> To solve this problem, in my opinion, RebalancePartitioner should use its own operator id for the initial value.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)