rhauch commented on a change in pull request #8720:
URL: https://github.com/apache/kafka/pull/8720#discussion_r431395579



##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/ErrorReporter.java
##########
@@ -16,17 +16,25 @@
  */
 package org.apache.kafka.connect.runtime.errors;
 
+import org.apache.kafka.clients.producer.RecordMetadata;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+
 /**
  * Report an error using the information contained in the {@link 
ProcessingContext}.
  */
 public interface ErrorReporter extends AutoCloseable {
 
     /**
-     * Report an error.
+     * Report an error and return the producer future.
      *
      * @param context the processing context (cannot be null).
+     * @return future result from the producer sending a record to Kafka.
      */
-    void report(ProcessingContext context);
+    default Future<RecordMetadata> report(ProcessingContext context) {
+        return CompletableFuture.completedFuture(null);
+    }

Review comment:
       We should not make this method a default method, since both 
implementations of the interface define this method.

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/LogReporter.java
##########
@@ -50,17 +53,18 @@ public LogReporter(ConnectorTaskId id, ConnectorConfig 
connConfig, ErrorHandling
      * @param context the processing context.
      */
     @Override
-    public void report(ProcessingContext context) {
+    public Future<RecordMetadata> report(ProcessingContext context) {
         if (!connConfig.enableErrorLog()) {
-            return;
+            return CompletableFuture.completedFuture(null);
         }
 
         if (!context.failed()) {
-            return;
+            return CompletableFuture.completedFuture(null);
         }
 
         log.error(message(context), context.error());
         errorHandlingMetrics.recordErrorLogged();
+        return CompletableFuture.completedFuture(null);

Review comment:
       What about creating:
   ```
       private static final Future<RecordMetadata> COMPLETED = 
CompletableFuture.completedFuture(null);
   ```
   
   and then returning that instance in all of these places. Since it's already 
completed, immutable, and we don't allow cancellation, it should be fine to 
reuse in this `LogReporter`.

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/WorkerErrantRecordReporter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.kafka.connect.runtime.errors;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.runtime.InternalSinkRecord;
+import org.apache.kafka.connect.sink.ErrantRecordReporter;
+import org.apache.kafka.connect.sink.SinkRecord;
+
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.HeaderConverter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class WorkerErrantRecordReporter implements ErrantRecordReporter {
+
+    private static final Logger log = 
LoggerFactory.getLogger(WorkerErrantRecordReporter.class);
+
+    private RetryWithToleranceOperator retryWithToleranceOperator;
+    private Converter keyConverter;
+    private Converter valueConverter;
+    private HeaderConverter headerConverter;
+
+    // Visible for testing
+    final LinkedList<Future<Void>> futures;
+
+    public WorkerErrantRecordReporter(
+        RetryWithToleranceOperator retryWithToleranceOperator,
+        Converter keyConverter,
+        Converter valueConverter,
+        HeaderConverter headerConverter
+    ) {
+        this.retryWithToleranceOperator = retryWithToleranceOperator;
+        this.keyConverter = keyConverter;
+        this.valueConverter = valueConverter;
+        this.headerConverter = headerConverter;
+        this.futures = new LinkedList<>();
+    }
+
+    @Override
+    public Future<Void> report(SinkRecord record, Throwable error) {
+        ConsumerRecord<byte[], byte[]> consumerRecord;
+
+        // Most of the records will be an internal sink record, but the task 
could potentially
+        // report modified or new records, so handle both cases
+        if (record instanceof InternalSinkRecord) {
+            consumerRecord = ((InternalSinkRecord) record).originalRecord();
+        } else {
+            String topic = record.topic();

Review comment:
       How about clarifying this a bit:
   ```suggestion
               // Generate a new consumer record from the modified sink record. 
We prefer
               // to send the original consumer record (pre-transformed) to the 
DLQ, 
               // but in this case we don't have one and send the potentially 
transformed
               // record instead
               String topic = record.topic();
   ```

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/WorkerErrantRecordReporter.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.kafka.connect.runtime.errors;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.runtime.InternalSinkRecord;
+import org.apache.kafka.connect.sink.ErrantRecordReporter;
+import org.apache.kafka.connect.sink.SinkRecord;
+
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.HeaderConverter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class WorkerErrantRecordReporter implements ErrantRecordReporter {
+
+    private static final Logger log = 
LoggerFactory.getLogger(WorkerErrantRecordReporter.class);
+
+    private RetryWithToleranceOperator retryWithToleranceOperator;
+    private Converter keyConverter;
+    private Converter valueConverter;
+    private HeaderConverter headerConverter;
+
+    // Visible for testing
+    final LinkedList<Future<Void>> futures;
+
+    public WorkerErrantRecordReporter(
+        RetryWithToleranceOperator retryWithToleranceOperator,
+        Converter keyConverter,
+        Converter valueConverter,
+        HeaderConverter headerConverter
+    ) {
+        this.retryWithToleranceOperator = retryWithToleranceOperator;
+        this.keyConverter = keyConverter;
+        this.valueConverter = valueConverter;
+        this.headerConverter = headerConverter;
+        this.futures = new LinkedList<>();
+    }
+
+    @Override
+    public Future<Void> report(SinkRecord record, Throwable error) {
+        ConsumerRecord<byte[], byte[]> consumerRecord;
+
+        // Most of the records will be an internal sink record, but the task 
could potentially
+        // report modified or new records, so handle both cases
+        if (record instanceof InternalSinkRecord) {
+            consumerRecord = ((InternalSinkRecord) record).originalRecord();
+        } else {
+            String topic = record.topic();
+            byte[] key = keyConverter.fromConnectData(topic, 
record.keySchema(), record.key());
+            byte[] value = valueConverter.fromConnectData(topic,
+                record.valueSchema(), record.value());
+
+            RecordHeaders headers = new RecordHeaders();
+            if (record.headers() != null) {
+                for (Header header : record.headers()) {
+                    String headerKey = header.key();
+                    byte[] rawHeader = 
headerConverter.fromConnectHeader(topic, headerKey,
+                        header.schema(), header.value());
+                    headers.add(headerKey, rawHeader);
+                }
+            }
+
+            consumerRecord = new ConsumerRecord<>(record.topic(), 
record.kafkaPartition(),
+                record.kafkaOffset(), record.timestamp(), 
record.timestampType(), -1L, -1,
+                -1, key, value, headers);

Review comment:
       We should use the length of the key and value in the record:
   ```suggestion
               int keyLength = key != null ? key.length : -1;
               int valLength = value != null ? value.length : -1;
               consumerRecord = new ConsumerRecord<>(record.topic(), 
record.kafkaPartition(),
                   record.kafkaOffset(), record.timestamp(), 
record.timestampType(), -1L, keyLength,
                   valLength, key, value, headers);
   ```

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
##########
@@ -360,6 +364,10 @@ private void doCommit(Map<TopicPartition, 
OffsetAndMetadata> offsets, boolean cl
     }
 
     private void commitOffsets(long now, boolean closing) {
+        if (workerErrantRecordReporter != null) {
+            workerErrantRecordReporter.awaitAllFutures();

Review comment:
       Once again, please add trace log messages before an after this line.




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