becketqin commented on a change in pull request #8535: [FLINK-11693] Add
KafkaSerializationSchema that uses ProducerRecord
URL: https://github.com/apache/flink/pull/8535#discussion_r296125913
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
##########
@@ -481,27 +508,140 @@ public FlinkKafkaProducer(
* round-robin fashion.
* @param semantic Defines semantic that will be used by this producer
(see {@link FlinkKafkaProducer.Semantic}).
* @param kafkaProducersPoolSize Overwrite default KafkaProducers pool
size (see {@link FlinkKafkaProducer.Semantic#EXACTLY_ONCE}).
+ *
+ * @deprecated use {@link #FlinkKafkaProducer(String,
KafkaSerializationSchema, Properties, FlinkKafkaProducer.Semantic)}
+ */
+ @Deprecated
+ public FlinkKafkaProducer(
+ String defaultTopicId,
+ KeyedSerializationSchema<IN> serializationSchema,
+ Properties producerConfig,
+ Optional<FlinkKafkaPartitioner<IN>> customPartitioner,
+ FlinkKafkaProducer.Semantic semantic,
+ int kafkaProducersPoolSize) {
+ this(
+ defaultTopicId,
+ serializationSchema,
+ customPartitioner.orElse(null),
+ null, /* kafka serialization schema */
+ producerConfig,
+ semantic,
+ kafkaProducersPoolSize);
+ }
+
+ /**
+ * Creates a {@link FlinkKafkaProducer} for a given topic. The sink
produces its input to
+ * the topic. It accepts a keyed {@link KafkaSerializationSchema} for
serializing records to
+ * a {@link ProducerRecord}, including partitioning information.
+ *
+ * @param defaultTopicId The default topic to write data to
+ * @param serializationSchema A serializable serialization schema for
turning user objects into a kafka-consumable byte[] supporting key/value
messages
+ * @param producerConfig Configuration properties for the
KafkaProducer. 'bootstrap.servers.' is the only required argument.
+ * @param semantic Defines semantic that will be used by this producer
(see {@link FlinkKafkaProducer.Semantic}).
+ */
+ public FlinkKafkaProducer(
+ String defaultTopicId,
+ KafkaSerializationSchema<IN> serializationSchema,
+ Properties producerConfig,
+ FlinkKafkaProducer.Semantic semantic) {
+ this(
+ defaultTopicId,
+ serializationSchema,
+ producerConfig,
+ semantic,
+ DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
+ }
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces
its input to
+ * the topic. It accepts a keyed {@link KafkaSerializationSchema} and
possibly a custom {@link FlinkKafkaPartitioner}.
+ *
+ * @param defaultTopicId The default topic to write data to
+ * @param serializationSchema A serializable serialization schema for
turning user objects into a kafka-consumable byte[] supporting key/value
messages
+ * @param producerConfig Configuration properties for the
KafkaProducer. 'bootstrap.servers.' is the only required argument.
+ * @param semantic Defines semantic that will be used by this producer
(see {@link FlinkKafkaProducer.Semantic}).
+ * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool
size (see {@link FlinkKafkaProducer.Semantic#EXACTLY_ONCE}).
*/
public FlinkKafkaProducer(
String defaultTopicId,
- KeyedSerializationSchema<IN> serializationSchema,
+ KafkaSerializationSchema<IN> serializationSchema,
Properties producerConfig,
- Optional<FlinkKafkaPartitioner<IN>> customPartitioner,
FlinkKafkaProducer.Semantic semantic,
int kafkaProducersPoolSize) {
+ this(
+ defaultTopicId,
+ null, null, /* keyed schema and
FlinkKafkaPartitioner */
+ serializationSchema,
+ producerConfig,
+ semantic,
+ kafkaProducersPoolSize);
+ }
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces
its input to
+ * the topic. It accepts a keyed {@link KafkaSerializationSchema} and
possibly a custom {@link FlinkKafkaPartitioner}.
+ *
+ * <p>If a partitioner is not provided, written records will be
partitioned by the attached key of each
+ * record (as determined by {@link
KeyedSerializationSchema#serializeKey(Object)}). If written records do not
+ * have a key (i.e., {@link
KeyedSerializationSchema#serializeKey(Object)} returns {@code null}), they
+ * will be distributed to Kafka partitions in a round-robin fashion.
+ *
+ * @param defaultTopicId The default topic to write data to
+ * @param keyedSchema A serializable serialization schema for turning
user objects into a kafka-consumable byte[] supporting key/value messages
+ * @param customPartitioner A serializable partitioner for assigning
messages to Kafka partitions.
+ * If a partitioner is not provided, records
will be partitioned by the key of each record
+ * (determined by {@link
KeyedSerializationSchema#serializeKey(Object)}). If the keys
+ * are {@code null}, then records will be
distributed to Kafka partitions in a
+ * round-robin fashion.
+ * @param kafkaSchema A serializable serialization schema for turning
user objects into a kafka-consumable byte[] supporting key/value messages
+ * @param producerConfig Configuration properties for the
KafkaProducer. 'bootstrap.servers.' is the only required argument.
+ * @param semantic Defines semantic that will be used by this producer
(see {@link FlinkKafkaProducer.Semantic}).
+ * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool
size (see {@link FlinkKafkaProducer.Semantic#EXACTLY_ONCE}).
+ */
+ private FlinkKafkaProducer(
+ String defaultTopicId,
+ KeyedSerializationSchema<IN> keyedSchema,
+ FlinkKafkaPartitioner<IN> customPartitioner,
+ KafkaSerializationSchema<IN> kafkaSchema,
+ Properties producerConfig,
+ FlinkKafkaProducer.Semantic semantic,
+ int kafkaProducersPoolSize) {
super(new FlinkKafkaProducer.TransactionStateSerializer(), new
FlinkKafkaProducer.ContextStateSerializer());
this.defaultTopicId = checkNotNull(defaultTopicId,
"defaultTopicId is null");
- this.schema = checkNotNull(serializationSchema,
"serializationSchema is null");
+
+ if (kafkaSchema != null) {
+ this.keyedSchema = null;
+ this.kafkaSchema = kafkaSchema;
+ this.flinkKafkaPartitioner = null;
+ ClosureCleaner.clean(
+ this.kafkaSchema,
ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+
+ if (customPartitioner != null) {
Review comment:
I had an earlier comment about this restriction. It seems the current
interface of `KafkaSerializationSchema` cannot fully cover the functionality of
the partitioner, because the implementations of `KafkaSerializationSchema` do
not have enough information, e.g. number of partitions, task parallelisms, etc.
Should we add those information to the arguments of
`KafkaSerializationSchema.serialize()`?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services