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:
[email protected]


With regards,
Apache Git Services

Reply via email to