fapaul commented on a change in pull request #16769:
URL: https://github.com/apache/flink/pull/16769#discussion_r686877239



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.table;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import 
org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
+import 
org.apache.flink.streaming.connectors.kafka.sink.KafkaRecordSerializationSchema;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.kafka.clients.producer.ProducerRecord;
+
+import javax.annotation.Nullable;
+
+/**
+ * SerializationSchema used by {@link KafkaDynamicSink} to configure a {@link
+ * org.apache.flink.streaming.connectors.kafka.sink.KafkaSink}.
+ */
+class DynamicKafkaRecordSerializationSchema implements 
KafkaRecordSerializationSchema<RowData> {
+
+    private final String topic;
+    private final FlinkKafkaPartitioner<RowData> partitioner;
+    private final SerializationSchema<RowData> keySerialization;
+    private final SerializationSchema<RowData> valueSerialization;
+    private final RowData.FieldGetter[] keyFieldGetters;
+    private final RowData.FieldGetter[] valueFieldGetters;
+    private final boolean hasMetadata;
+    private final int[] metadataPositions;
+    private final boolean upsertMode;
+
+    private transient boolean isPartitionerOpen = false;
+
+    DynamicKafkaRecordSerializationSchema(
+            String topic,
+            @Nullable FlinkKafkaPartitioner<RowData> partitioner,
+            @Nullable SerializationSchema<RowData> keySerialization,
+            SerializationSchema<RowData> valueSerialization,
+            RowData.FieldGetter[] keyFieldGetters,
+            RowData.FieldGetter[] valueFieldGetters,
+            boolean hasMetadata,
+            int[] metadataPositions,
+            boolean upsertMode) {
+        if (upsertMode) {
+            Preconditions.checkArgument(
+                    keySerialization != null && keyFieldGetters.length > 0,
+                    "Key must be set in upsert mode for serialization 
schema.");
+        }
+        this.topic = topic;
+        this.partitioner = partitioner;
+        this.keySerialization = keySerialization;
+        this.valueSerialization = valueSerialization;
+        this.keyFieldGetters = keyFieldGetters;
+        this.valueFieldGetters = valueFieldGetters;
+        this.hasMetadata = hasMetadata;
+        this.metadataPositions = metadataPositions;
+        this.upsertMode = upsertMode;
+    }
+
+    @Override
+    public ProducerRecord<byte[], byte[]> serialize(
+            RowData consumedRow, KafkaSinkContext context, Long timestamp) {
+        if (partitioner != null && !isPartitionerOpen) {
+            partitioner.open(
+                    context.getParallelInstanceId(), 
context.getNumberOfParallelInstances());
+            isPartitionerOpen = true;
+        }
+        // shortcut in case no input projection is required
+        if (keySerialization == null && !hasMetadata) {
+            final byte[] valueSerialized = 
valueSerialization.serialize(consumedRow);
+            return new ProducerRecord<>(
+                    topic,
+                    extractPartition(
+                            consumedRow,
+                            null,
+                            valueSerialized,
+                            context.getPartitionsForTopic(topic)),
+                    null,
+                    valueSerialized);
+        }
+        final byte[] keySerialized;
+        if (keySerialization == null) {
+            keySerialized = null;
+        } else {
+            final RowData keyRow = createProjectedRow(consumedRow, 
RowKind.INSERT, keyFieldGetters);
+            keySerialized = keySerialization.serialize(keyRow);
+        }
+
+        final byte[] valueSerialized;
+        final RowKind kind = consumedRow.getRowKind();
+        final RowData valueRow =
+                DynamicKafkaRecordSerializationSchema.createProjectedRow(
+                        consumedRow, kind, valueFieldGetters);

Review comment:
       Not this is correct because the only case where we do not serialize for 
`DELETE` and `UPDATE_BEFORE` rows is if the sink is on upsert Mode. 
   
   Do you think it is worth it to complicate the branching?
   Btw the old one 
https://github.com/apache/flink/blob/f22b6ddc08dcd046458b4aeebfe828073410952a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaSerializationSchema.java#L127
 is doing the same stuff




-- 
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]


Reply via email to