You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by nielsbasjes <gi...@git.apache.org> on 2018/06/18 19:54:26 UTC
[GitHub] flink pull request #6181: [FLINK-9610] [flink-connector-kafka-base] Add Kafk...
GitHub user nielsbasjes opened a pull request:
https://github.com/apache/flink/pull/6181
[FLINK-9610] [flink-connector-kafka-base] Add Kafka Partitioner that uses the hash of the provided key.
## What is the purpose of the change
Add the simple feature of being able to route records into Kafka using a key based partitioning.
## Brief change log
- Added the FlinkKeyHashPartitioner class with some tests.
## Verifying this change
This change added tests and can be verified as follows:
- Use this instead of the FlinkFixedPartitioner while instantiating the FlinkKafkaProducer. Also add an KeyedSerializationSchema implementation that returns the right key that is to be used.
## Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): no
- The public API, i.e., is any changed class annotated with `@Public(Evolving)`: yes
- The serializers: no
- The runtime per-record code paths (performance sensitive): no
- Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: no
- The S3 file system connector: no
## Documentation
- Does this pull request introduce a new feature? yes
- If yes, how is the feature documented? JavaDocs
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/nielsbasjes/flink KafkaKeyHashPartitioner
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/6181.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #6181
----
commit 22f814cab0710653ec1766531db7077f9e8fd534
Author: Niels Basjes <nb...@...>
Date: 2018-06-18T19:16:22Z
[FLINK-9610] Add Kafka Partitioner that uses the hash of the provided key
----
---
[GitHub] flink pull request #6181: [FLINK-9610] [flink-connector-kafka-base] Add Kafk...
Posted by nielsbasjes <gi...@git.apache.org>.
Github user nielsbasjes commented on a diff in the pull request:
https://github.com/apache/flink/pull/6181#discussion_r196435400
--- 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 --
Yes, good catch. I fixed that just now.
---
[GitHub] flink pull request #6181: [FLINK-9610] [flink-connector-kafka-base] Add Kafk...
Posted by tedyu <gi...@git.apache.org>.
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()) ?
---