AHeise commented on a change in pull request #16783:
URL: https://github.com/apache/flink/pull/16783#discussion_r690486505



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/sink/KafkaRecordSerializationSchemaBuilder.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
+
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.Configurable;
+import org.apache.kafka.common.serialization.Serializer;
+
+import javax.annotation.Nullable;
+
+import java.util.Map;
+import java.util.OptionalInt;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Builder to construct {@link KafkaRecordSerializationSchema}.
+ *
+ * <p>This class should give a first entrypoint when trying to serialize 
elements to {@link
+ * ProducerRecord}. The following examples show some of the possibilities.
+ *
+ * <pre>Simple key-value serialization:
+ * {@code
+ * new KafkaRecordSerializationSchemaBuilder<String>("topic")
+ *     .withKeySerializationSchema(new SimpleStringSchema())
+ *     .withValueSerializationSchema(new SimpleStringSchema())
+ *     .build()
+ * }</pre>
+ *
+ * <pre>Using Kafka's serialization stack:
+ * {@code
+ * new KafkaRecordSerializationSchemaBuilder<String>("topic")
+ *     .withKeySerializer(StringSerializer.class)
+ *     .withValueSerializer(StringSerializer.class)
+ *     .build()
+ * }</pre>
+ *
+ * <pre>With custom partitioner:
+ * {@code
+ * new KafkaRecordSerializationSchemaBuilder<String>("topic")
+ *     .withPartitioner(MY_FLINK_PARTITIONER)
+ *     .withKeySerializationSchema(StringSerializer.class)
+ *     .build()
+ * }</pre>
+ *
+ * <p>The different serialization methods for key and value are mutually 
exclusive thus i.e. it is
+ * not possible to use {@link 
#withKeySerializationSchema(SerializationSchema)} and {@link
+ * #withKeySerializer(Class)} on the same builder instance.
+ *
+ * <p>It is necessary to configure exactly one serialization method for the 
value.
+ *
+ * @param <IN> type of records to be serialized
+ * @see KafkaRecordSerializationSchema#builder(String)
+ */
+public class KafkaRecordSerializationSchemaBuilder<IN> {
+
+    @Nullable private final String topic;
+    @Nullable private final Function<IN, String> topicSelector;
+    @Nullable private FlinkKafkaPartitioner<IN> partitioner;
+    @Nullable private SerializationSchema<IN> keySerializationSchema;
+
+    private SerializationSchema<IN> valueSerializationSchema;
+
+    KafkaRecordSerializationSchemaBuilder(String topic) {
+        this.topic = checkNotNull(topic);
+        this.topicSelector = null;
+    }
+
+    KafkaRecordSerializationSchemaBuilder(Function<IN, String> topicSupplier) {
+        this.topicSelector = checkNotNull(topicSupplier);
+        this.topic = null;
+    }
+
+    /**
+     * Sets a custom partitioner determining the target partition of the 
target topic.
+     *
+     * @param partitioner
+     * @return {@link KafkaRecordSerializationSchemaBuilder}
+     */
+    public KafkaRecordSerializationSchemaBuilder<IN> withPartitioner(
+            FlinkKafkaPartitioner<IN> partitioner) {
+        this.partitioner = checkNotNull(partitioner);
+        return this;
+    }
+
+    /**
+     * Sets a {@link SerializationSchema} which is used to serialize the 
incoming element to the key
+     * of the {@link ProducerRecord}.
+     *
+     * @param keySerializationSchema
+     * @return {@link KafkaRecordSerializationSchemaBuilder}
+     */
+    public KafkaRecordSerializationSchemaBuilder<IN> 
withKeySerializationSchema(
+            SerializationSchema<IN> keySerializationSchema) {
+        checkState(this.keySerializationSchema == null, 
exceedsKeySerializerConfigurations());
+        this.keySerializationSchema = checkNotNull(keySerializationSchema);
+        return this;
+    }
+
+    /**
+     * Sets Kafka's {@link Serializer} to serialize incoming elements to the 
key of the {@link
+     * ProducerRecord}.
+     *
+     * @param keySerializer
+     * @return {@link KafkaRecordSerializationSchemaBuilder}
+     */
+    public KafkaRecordSerializationSchemaBuilder<IN> withKeySerializer(
+            Class<? extends Serializer<IN>> keySerializer) {
+        checkState(keySerializationSchema == null, 
exceedsKeySerializerConfigurations());
+        this.keySerializationSchema =
+                new KafkaSerializerWrapper<>(keySerializer, topic, 
topicSelector);
+        return this;
+    }
+
+    /**
+     * Sets a configurable Kafka {@link Serializer} and pass a configuration 
to serialize incoming
+     * elements to the key of the {@link ProducerRecord}.
+     *
+     * @param keySerializerWithConfiguration
+     * @param configuration
+     * @param <T> type of the used serializer class
+     * @return {@link KafkaRecordSerializationSchemaBuilder}
+     */
+    public <T extends Configurable & Serializer<IN>>
+            KafkaRecordSerializationSchemaBuilder<IN> 
withConfigurableKeySerializer(
+                    Class<T> keySerializerWithConfiguration, Map<String, 
String> configuration) {
+        checkState(keySerializationSchema == null, 
exceedsKeySerializerConfigurations());
+        this.keySerializationSchema =
+                new KafkaSerializerWrapper<>(
+                        keySerializerWithConfiguration, configuration, topic, 
topicSelector);
+        return this;
+    }
+
+    /**
+     * Sets a {@link SerializationSchema} which is used to serialize the 
incoming element to the
+     * value of the {@link ProducerRecord}.
+     *
+     * @param valueSerializationSchema
+     * @return {@link KafkaRecordSerializationSchemaBuilder}
+     */
+    public KafkaRecordSerializationSchemaBuilder<IN> 
withValueSerializationSchema(
+            SerializationSchema<IN> valueSerializationSchema) {
+        checkState(this.valueSerializationSchema == null, 
exceedsValueSerializerConfigurations());
+        this.valueSerializationSchema = checkNotNull(valueSerializationSchema);
+        return this;
+    }

Review comment:
       The ugliness that is called explicit type parameter in a method call. 
You can't tell me that you enjoy invoking a method `ClassA.<type>methodB` and 
prefer it over `ClassA.methodB`.




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to