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

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

zhijiangW commented on a change in pull request #7051: [FLINK-10820][network] 
Simplify the RebalancePartitioner implementation
URL: https://github.com/apache/flink/pull/7051#discussion_r232553672
 
 

 ##########
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
 ##########
 @@ -33,29 +31,14 @@
 public class RebalancePartitioner<T> extends StreamPartitioner<T> {
        private static final long serialVersionUID = 1L;
 
-       private final int[] returnArray = {Integer.MAX_VALUE - 1};
+       private final int[] returnArray = new int[] { -1 };
 
        @Override
-       public int[] selectChannels(
-                       SerializationDelegate<StreamRecord<T>> record,
-                       int numChannels) {
-               int newChannel = ++returnArray[0];
-               if (newChannel >= numChannels) {
-                       returnArray[0] = resetValue(numChannels, newChannel);
-               }
+       public int[] selectChannels(SerializationDelegate<StreamRecord<T>> 
record, int numChannels) {
+               returnArray[0] = (returnArray[0] + 1) % numChannels;
                return returnArray;
        }
 
-       private static int resetValue(
-                       int numChannels,
-                       int newChannel) {
-               if (newChannel == Integer.MAX_VALUE) {
-                       // Initializes the first partition, this branch is only 
entered when initializing.
-                       return ThreadLocalRandom.current().nextInt(numChannels);
 
 Review comment:
   Thanks for the good reminding. I forgot to trace the history issue of 
introducing random here, and it is somewhat necessary to keep random to avoid 
all the parallelism tasks starting with the same index and keep the same rate 
to emit record, which would cause unbalanced case.
   
   Actually I have two motivations for this changes:
   1. If keeping the same logic with current `RoundRobinChannelSelector`, they 
may be integrated into one in the future.
   2. The current implementation is a little hacky because it relies on the 
special `Integer.MAX_VALUE` to realize random way, so resulting in another 
extra condition check `if (newChannel == Integer.MAX_VALUE)` for next 
round-robin. Especially for the corner case of `numChannels=Integer.MAX_VALUE`, 
it will trigger next random index after selecting the last channel index. 
Actually the random index should be triggered only once at the first time.
   
   Only considering the second reason currently, I would update the codes for 
that. :)

----------------------------------------------------------------
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:
us...@infra.apache.org


> Simplify the RebalancePartitioner implementation
> ------------------------------------------------
>
>                 Key: FLINK-10820
>                 URL: https://issues.apache.org/jira/browse/FLINK-10820
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Network
>    Affects Versions: 1.8.0
>            Reporter: zhijiang
>            Assignee: zhijiang
>            Priority: Minor
>              Labels: pull-request-available
>
> The current {{RebalancePartitioner}} implementations seems a little hacky for 
> selecting a random number as the first channel index, and the following 
> selections based on this random index in round-robin fashion.
> We can define a constant as the first channel index to make the 
> implementation simple and readable. To do so, it will not change the 
> rebalance semantics.



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

Reply via email to