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 2015/08/27 14:22:46 UTC

[jira] [Commented] (FLINK-1725) New Partitioner for better load balancing for skewed data

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

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

Github user mbalassi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1069#discussion_r38088713
  
    --- Diff: flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/partitioner/PartialPartitioner.java ---
    @@ -0,0 +1,60 @@
    +/*
    + * 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.functions.KeySelector;
    +import org.apache.flink.runtime.plugable.SerializationDelegate;
    +import org.apache.flink.streaming.api.streamrecord.StreamRecord;
    +import com.google.common.hash.HashFunction;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Partitioner that map each key on any two channels using power of two choices.
    + *
    + * @param <T>
    + *            Type of the Tuple
    + */
    +public class PartialPartitioner<T> extends StreamPartitioner<T> {
    +	private static final long serialVersionUID = 1L;
    +
    +	private long[] targetTaskStats; // maintain past history of forwarded messages
    +	private HashFunction h1 = Hashing.murmur3_128(13);
    +	private HashFunction h2 = Hashing.murmur3_128(17);
    +	KeySelector<T, ?> keySelector;
    +	private int[] returnArray = new int[1];
    +
    +	public PartialPartitioner(KeySelector<T, ?> keySelector, int numberOfOutputChannels) {
    +		super(PartitioningStrategy.PARTIAL);
    +		this.targetTaskStats = new long[numberOfOutputChannels];
    +		this.keySelector = keySelector;
    +	}
    +
    +	@Override
    +	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
    +			int numberOfOutputChannels) {
    +		String str = record.getInstance().getKey(keySelector).toString(); // assume key is the first field
    --- End diff --
    
    Comment is copy-pasted from the Storm [implementation](https://github.com/gdfm/partial-key-grouping/blob/master/src/main/java/com/yahoo/labs/slb/PartialKeyGrouping.java), please remove as does not apply for Flink. :)


> New Partitioner for better load balancing for skewed data
> ---------------------------------------------------------
>
>                 Key: FLINK-1725
>                 URL: https://issues.apache.org/jira/browse/FLINK-1725
>             Project: Flink
>          Issue Type: Improvement
>          Components: New Components
>    Affects Versions: 0.8.1
>            Reporter: Anis Nasir
>            Assignee: Anis Nasir
>              Labels: LoadBalancing, Partitioner
>   Original Estimate: 336h
>  Remaining Estimate: 336h
>
> Hi,
> We have recently studied the problem of load balancing in Storm [1].
> In particular, we focused on key distribution of the stream for skewed data.
> We developed a new stream partitioning scheme (which we call Partial Key Grouping). It achieves better load balancing than key grouping while being more scalable than shuffle grouping in terms of memory.
> In the paper we show a number of mining algorithms that are easy to implement with partial key grouping, and whose performance can benefit from it. We think that it might also be useful for a larger class of algorithms.
> Partial key grouping is very easy to implement: it requires just a few lines of code in Java when implemented as a custom grouping in Storm [2].
> For all these reasons, we believe it will be a nice addition to the standard Partitioners available in Flink. If the community thinks it's a good idea, we will be happy to offer support in the porting.
> References:
> [1]. https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream-processing-engines.pdf
> [2]. https://github.com/gdfm/partial-key-grouping



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)