AHeise commented on a change in pull request #16783:
URL: https://github.com/apache/flink/pull/16783#discussion_r687670107
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/sink/KafkaRecordSerializationSchema.java
##########
@@ -82,4 +87,97 @@ default void open(SerializationSchema.InitializationContext
context) throws Exce
*/
int[] getPartitionsForTopic(String topic);
}
+
+ /**
+ * Convenience wrapper to write value-only to Kafka with a derived topic.
+ *
+ * @param topicSupplier to derive a topic from the incoming elements
Review comment:
`topicSelector`?
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/sink/KafkaRecordSerializationSchema.java
##########
@@ -82,4 +87,97 @@ default void open(SerializationSchema.InitializationContext
context) throws Exce
*/
int[] getPartitionsForTopic(String topic);
}
+
+ /**
+ * Convenience wrapper to write value-only to Kafka with a derived topic.
+ *
+ * @param topicSupplier to derive a topic from the incoming elements
+ * @param serializationSchema to serialize the incoming elements to bytes
+ * @param <T> type of the values being serialized
+ * @return A {@link KafkaRecordSerializationSchema} which serializes a
{@link ProducerRecord}
+ * with only a value and a derived topic.
+ */
+ static <T> KafkaRecordSerializationSchema<T> valueOnly(
+ Function<T, String> topicSupplier, SerializationSchema<T>
serializationSchema) {
+ return new KafkaValueOnlyRecordSerializationSchema<>(topicSupplier,
serializationSchema);
+ }
+
+ /**
+ * Convenience wrapper to write value-only to Kafka with a fixed topic.
+ *
+ * @param topic which used as destination for all the records
+ * @param serializationSchema to serialize the incoming elements to bytes
+ * @param <T> type of the values being serialized
+ * @return A {@link KafkaRecordSerializationSchema} which serializes a
{@link ProducerRecord}
+ * with only a value and a fixed topic.
+ */
+ static <T> KafkaRecordSerializationSchema<T> valueOnly(
+ String topic, SerializationSchema<T> serializationSchema) {
+ return valueOnly((element) -> topic, serializationSchema);
+ }
+
+ /**
+ * Schema wrapper to write value-only to Kafka with a fixed topic. It
supports Kafka's
+ * serialization stack by serialization with {@link Serializer}.
+ *
+ * @param topicSupplier to derive a topic from the incoming elements
+ * @param serializerClass Kafka's {@link Serializer} to convert the
incoming elements tu bytes
+ * @param <T> type of the values being serialized
+ * @return A {@link KafkaRecordSerializationSchema} which serializes a
{@link ProducerRecord}
+ * with only a value and a fixed topic.
+ */
+ static <T> KafkaRecordSerializationSchema<T> valueOnly(
+ Function<T, String> topicSupplier, Class<? extends Serializer<T>>
serializerClass) {
Review comment:
I think that this is only the start of the story. Folks will ask if we
can also support a Serializer for Key and maybe the partitioner?
At that time we will use a return value that is specialized and will only
additional configuration.
`valueOnly(...).withKeySerializer(...).withPartitioner(...)`. Most of the
things can be done later but it's obvious that `valueOnly` would then be a
misfit.
I'd recommend to rename all valueOnly to some `forX`, like
`forKafkaSerializer`. I'm also thinking that we should only have two root
convenience methods now:
`forKafkaSerializer(Class<?> ).withConfiguration(Map<String, String>)` for
some kind of builder pattern (I'm also fine with replacing `withConfiguration`
with `setConfiguration`). The `build` method would then be
`.withTopicSelector/intoTopic`.
and a similar thing `forSerializationSchema`.
I know this is a bigger change to this PR and might not make it into 1.14
but it's better than having 7 deprecated methods in the near future.
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/sink/KafkaValueOnlySerializer.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.TemporaryClassLoaderContext;
+
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.Configurable;
+import org.apache.kafka.common.serialization.Serializer;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Function;
+
+/** A simple serialization schema to support Kafka's {@link Serializer}. */
+class KafkaValueOnlySerializer<IN> implements
KafkaRecordSerializationSchema<IN> {
Review comment:
`KafkaValueOnlyKafkaSerializer`? I had a hard time to understand the
difference between this and `KafkaValueOnlyRecordSerializationSchema`.
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]