[
https://issues.apache.org/jira/browse/FLINK-9979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16565152#comment-16565152
]
ASF GitHub Bot commented on FLINK-9979:
---------------------------------------
twalthr commented on a change in pull request #6440: [FLINK-9979] [table]
Support a FlinkKafkaPartitioner for Kafka table sink factory
URL: https://github.com/apache/flink/pull/6440#discussion_r206842240
##########
File path:
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java
##########
@@ -175,6 +183,69 @@ public Kafka startFromSpecificOffset(int partition, long
specificOffset) {
return this;
}
+ /**
+ * Configures how to partition records from Flink's partitions into
Kafka's partitions.
+ *
+ * <p>This strategy ensures that each Flink partition ends up in one
Kafka partition.
+ *
+ * <p>Note: One Kafka partition can contain multiple Flink partitions.
Examples:
+ *
+ * <p>More Flink partitions than Kafka partitions. Some (or all) Kafka
partitions contain
+ * the output of more than one flink partition:
+ * <pre>
+ * Flink Sinks Kafka Partitions
+ * 1 ----------------> 1
+ * 2 --------------/
+ * 3 -------------/
+ * 4 ------------/
+ * </pre>
+ *
+ *
+ * <p>Fewer Flink partitions than Kafka partitions:
+ * <pre>
+ * Flink Sinks Kafka Partitions
+ * 1 ----------------> 1
+ * 2 ----------------> 2
+ * 3
+ * 4
+ * 5
+ * </pre>
+ *
+ * @see
org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner
+ */
+ public Kafka sinkPartitionerFixed() {
+ sinkPartitioner = CONNECTOR_SINK_PARTITIONER_VALUE_FIXED;
+ sinkPartitionerClass = null;
+ return this;
+ }
+
+ /**
+ * Configures how to partition records from Flink's partitions into
Kafka's partitions.
+ *
+ * <p>This strategy ensures that records will be distributed to Kafka
partitions in a
+ * round-robin fashion.
+ *
+ * <p>Note: This strategy is useful to avoid an unbalanced
partitioning. However, it will
+ * cause a lot of network connections between all the Flink instances
and all the Kafka brokers.
+ */
+ public Kafka sinkPartitionerRoundRobin() {
+ sinkPartitioner = CONNECTOR_SINK_PARTITIONER_VALUE_ROUND_ROBIN;
+ sinkPartitionerClass = null;
+ return this;
+ }
+
+ /**
+ * Configures how to partition records from Flink's partitions into
Kafka's partitions.
+ *
+ * <p>This strategy allows for a custom partitioner by providing an
implementation
+ * of {@link FlinkKafkaPartitioner}.
+ */
+ public Kafka sinkPartitionerCustom(Class<? extends
FlinkKafkaPartitioner> partitionerClass) {
Review comment:
The goal was actually to have a single point of validation. Currently, the
validation happens when converting the class into string-properties.
If we would allow an instance here, this instance would need to be
serialized into a string and deserialized again. We can still allow an instance
in the future but for now I would not make it too complicated.
----------------------------------------------------------------
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]
> Support a custom FlinkKafkaPartitioner for a Kafka table sink factory
> ---------------------------------------------------------------------
>
> Key: FLINK-9979
> URL: https://issues.apache.org/jira/browse/FLINK-9979
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: Timo Walther
> Assignee: Timo Walther
> Priority: Major
> Labels: pull-request-available
>
> Currently, the Kafka table sink factory does not support a custom
> FlinkKafkaPartitioner. However, this is needed for many use cases.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)