rmahindra123 commented on a change in pull request #4021:
URL: https://github.com/apache/hudi/pull/4021#discussion_r756250522



##########
File path: 
hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/ConnectTransactionParticipant.java
##########
@@ -215,13 +215,17 @@ private void writeRecords() {
         try {
           SinkRecord record = buffer.peek();
           if (record != null
-              && record.kafkaOffset() >= 
ongoingTransactionInfo.getLastWrittenKafkaOffset()) {
+              && record.kafkaOffset() == 
ongoingTransactionInfo.getExpectedKafkaOffset()) {
             ongoingTransactionInfo.getWriter().writeRecord(record);
-            
ongoingTransactionInfo.setLastWrittenKafkaOffset(record.kafkaOffset() + 1);
-          } else if (record != null && record.kafkaOffset() < 
committedKafkaOffset) {
-            LOG.warn(String.format("Received a kafka record with offset %s 
prior to last committed offset %s for partition %s",
-                record.kafkaOffset(), 
ongoingTransactionInfo.getLastWrittenKafkaOffset(),
-                partition));
+            ongoingTransactionInfo.setExpectedKafkaOffset(record.kafkaOffset() 
+ 1);
+          } else if (record != null && record.kafkaOffset() > 
ongoingTransactionInfo.getExpectedKafkaOffset()) {
+            LOG.warn(String.format("Received a kafka record with offset %s 
above the next expected kafka offset %s for partition %s, "
+                    + "hence resetting the kafka offset to %s",
+                record.kafkaOffset(),
+                ongoingTransactionInfo.getExpectedKafkaOffset(),
+                partition,
+                ongoingTransactionInfo.getExpectedKafkaOffset()));
+            context.offset(partition, 
ongoingTransactionInfo.getExpectedKafkaOffset());
           }

Review comment:
       Good point, I also added some more log messages.




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