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

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

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

    https://github.com/apache/flink/pull/6181#discussion_r196368052
  
    --- Diff: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKeyHashPartitioner.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.connectors.kafka.partitioner;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +import org.apache.flink.util.Preconditions;
    +
    +import javax.annotation.Nullable;
    +
    +import java.util.Arrays;
    +
    +/**
    + * A partitioner that uses the hash of the provided key to distribute
    + * the values over the partitions as evenly as possible.
    + * This partitioner ensures that all records with the same key will be 
sent to
    + * the same Kafka partition.
    + *
    + * <p>Note that this will cause a lot of network connections to be created 
between
    + * all the Flink instances and all the Kafka brokers.
    + */
    +@PublicEvolving
    +public class FlinkKeyHashPartitioner<T> extends FlinkKafkaPartitioner<T> {
    +
    +   private static final long serialVersionUID = -2006468063065010594L;
    +
    +   @Override
    +   public int partition(T record, byte[] key, byte[] value, String 
targetTopic, int[] partitions) {
    +           Preconditions.checkArgument(
    +                   partitions != null && partitions.length > 0,
    +                   "Partitions of the target topic is empty.");
    +
    +           return partitions[hash(key) % partitions.length];
    --- End diff --
    
    Should we guard against hash(key) % partitions.length < 0 (in case someone 
overrides hash()) ?


> Add Kafka partitioner that uses the key to partition by
> -------------------------------------------------------
>
>                 Key: FLINK-9610
>                 URL: https://issues.apache.org/jira/browse/FLINK-9610
>             Project: Flink
>          Issue Type: New Feature
>          Components: Kafka Connector
>            Reporter: Niels Basjes
>            Assignee: Niels Basjes
>            Priority: Major
>
> The kafka connector package only contains the FlinkFixedPartitioner 
> implementation of the FlinkKafkaPartitioner.
> The most common usecase I have seen is the need to spread the records across 
> the Kafka partitions while keeping all messages with the same key together.
> I'll put up a pull request with a very simple implementation that should make 
> this a lot easier for others to use and extend.



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

Reply via email to