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



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaShuffleProducer.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.Internal;
+import 
org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.PropertiesUtil;
+
+import org.apache.kafka.clients.producer.ProducerRecord;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Properties;
+
+import static 
org.apache.flink.streaming.connectors.kafka.FlinkKafkaShuffle.PARTITION_NUMBER;
+
+/**
+ * Flink Kafka Shuffle Producer Function.
+ * It is different from {@link FlinkKafkaProducer} in the way handling 
elements and watermarks
+ */
+@Internal
+public class FlinkKafkaShuffleProducer<IN, KEY> extends FlinkKafkaProducer<IN> 
{
+       private final KafkaSerializer kafkaSerializer;
+       private final KeySelector<IN, KEY> keySelector;
+       private final int numberOfPartitions;
+
+       FlinkKafkaShuffleProducer(
+               String defaultTopicId,
+               TypeInformationSerializationSchema<IN> schema,
+               Properties props,
+               KeySelector<IN, KEY> keySelector,
+               Semantic semantic,
+               int kafkaProducersPoolSize) {
+               super(defaultTopicId, (element, timestamp) -> null, props, 
semantic, kafkaProducersPoolSize);
+
+               this.kafkaSerializer = new 
KafkaSerializer<>(schema.getSerializer());
+               this.keySelector = keySelector;
+
+               Preconditions.checkArgument(
+                       props.getProperty(PARTITION_NUMBER) != null,
+                       "Missing partition number for Kafka Shuffle");
+               numberOfPartitions = PropertiesUtil.getInt(props, 
PARTITION_NUMBER, Integer.MIN_VALUE);
+       }
+
+       /**
+        * This is the function invoked to handle each element.
+        * @param transaction transaction state;
+        *                    elements are written to Kafka in transactions to 
guarantee different level of data consistency
+        * @param next element to handle
+        * @param context context needed to handle the element
+        * @throws FlinkKafkaException
+        */
+       @Override
+       public void invoke(KafkaTransactionState transaction, IN next, Context 
context) throws FlinkKafkaException {
+               checkErroneous();
+
+               // write timestamp to Kafka if timestamp is available
+               Long timestamp = context.timestamp();
+
+               int[] partitions = getPartitions(transaction);
+               int partitionIndex;
+               try {
+                       partitionIndex = KeyGroupRangeAssignment
+                               
.assignKeyToParallelOperator(keySelector.getKey(next), partitions.length, 
partitions.length);
+               } catch (Exception e) {
+                       throw new RuntimeException("Fail to assign a partition 
number to record");
+               }
+
+               ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(
+                       defaultTopicId, partitionIndex, timestamp, null, 
kafkaSerializer.serializeRecord(next, timestamp));
+               pendingRecords.incrementAndGet();
+               transaction.producer.send(record, callback);
+       }
+
+       /**
+        * This is the function invoked to handle each watermark.
+        * @param transaction transaction state;
+        *                    watermark are written to Kafka (if needed) in 
transactions
+        * @param watermark watermark to handle
+        * @throws FlinkKafkaException
+        */
+       @Override
+       public void invoke(KafkaTransactionState transaction, Watermark 
watermark) throws FlinkKafkaException {
+               checkErroneous();
+
+               int[] partitions = getPartitions(transaction);
+               int subtask = getRuntimeContext().getIndexOfThisSubtask();
+
+               // broadcast watermark
+               for (int partition : partitions) {
+                       ProducerRecord<byte[], byte[]> record = new 
ProducerRecord<>(
+                               defaultTopicId, partition, 
watermark.getTimestamp(), null, kafkaSerializer.serializeWatermark(watermark, 
subtask));
+                       pendingRecords.incrementAndGet();
+                       transaction.producer.send(record, callback);
+               }
+       }
+
+       private int[] getPartitions(KafkaTransactionState transaction) {
+               int[] partitions = topicPartitionsMap.get(defaultTopicId);
+               if (partitions == null) {
+                       partitions = getPartitionsByTopic(defaultTopicId, 
transaction.producer);
+                       topicPartitionsMap.put(defaultTopicId, partitions);
+               }
+
+               Preconditions.checkArgument(partitions.length == 
numberOfPartitions);
+
+               return partitions;
+       }
+
+       /**
+        * Flink Kafka Shuffle Serializer.
+        */
+       public static final class KafkaSerializer<IN> implements Serializable {
+               public static final int TAG_REC_WITH_TIMESTAMP = 0;
+               public static final int TAG_REC_WITHOUT_TIMESTAMP = 1;
+               public static final int TAG_WATERMARK = 2;
+
+               private final TypeSerializer<IN> serializer;
+
+               private transient DataOutputSerializer dos;
+
+               KafkaSerializer(TypeSerializer<IN> serializer) {
+                       this.serializer = serializer;
+               }
+
+               /**
+                * Format: TAG, (timestamp), record.
+                */
+               byte[] serializeRecord(IN record, Long timestamp) {
+                       if (dos == null) {
+                               dos = new DataOutputSerializer(16);
+                       }
+
+                       try {
+                               if (timestamp == null) {
+                                       dos.writeInt(TAG_REC_WITHOUT_TIMESTAMP);

Review comment:
       My gut feeling is that we need to forward also some events, such as end 
of partition. 
   I'm also not entirely sure how to deal with checkpoint barriers (see general 
remark).
   In any case, we should use `StreamElementSerializer` instead of implementing 
it again. 

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SinkFunction.java
##########
@@ -52,6 +54,20 @@ default void invoke(IN value, Context context) throws 
Exception {
                invoke(value);
        }
 
+       /**
+        * This function is called for every watermark.
+        *
+        * <p>You have to override this method when implementing a {@code 
SinkFunction} to handle watermark.
+        * This method has to be used together with {@link StreamShuffleSink}
+        *
+        * @param watermark The watermark to handle.
+        * @throws Exception This method may throw exceptions. Throwing an 
exception will cause the operation
+        *                   to fail and may trigger recovery.
+        */
+       default void invoke(Watermark watermark) throws Exception {

Review comment:
       If we use a custom operator for sink/source, we can avoid modifying the 
public API of `SinkFunction` at all. Stephan also talked about operators in the 
jira ticket.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to