1
0

[HUDI-2671] Fix kafka offset handling in Kafka Connect protocol (#4021)

Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local>
This commit is contained in:
rmahindra123
2021-11-24 10:03:58 -08:00
committed by GitHub
parent 9af219b7c1
commit 90f2ea2f12
4 changed files with 147 additions and 118 deletions

View File

@@ -149,7 +149,7 @@ public class ConnectTransactionParticipant implements TransactionParticipant {
LOG.info("Started a new transaction after receiving START_COMMIT for commit " + currentCommitTime);
try {
ongoingTransactionInfo = new TransactionInfo<>(currentCommitTime, writerProvider.getWriter(currentCommitTime));
ongoingTransactionInfo.setLastWrittenKafkaOffset(committedKafkaOffset);
ongoingTransactionInfo.setExpectedKafkaOffset(committedKafkaOffset);
} catch (Exception exception) {
LOG.warn("Error received while starting a new transaction", exception);
}
@@ -188,7 +188,7 @@ public class ConnectTransactionParticipant implements TransactionParticipant {
.setParticipantInfo(
ControlMessage.ParticipantInfo.newBuilder()
.setWriteStatus(KafkaConnectUtils.buildWriteStatuses(writeStatuses))
.setKafkaOffset(ongoingTransactionInfo.getLastWrittenKafkaOffset())
.setKafkaOffset(ongoingTransactionInfo.getExpectedKafkaOffset())
.build()
).build();
@@ -201,9 +201,9 @@ public class ConnectTransactionParticipant implements TransactionParticipant {
}
private void handleAckCommit(ControlMessage message) {
// Update lastKafkCommitedOffset locally.
if (ongoingTransactionInfo != null && committedKafkaOffset < ongoingTransactionInfo.getLastWrittenKafkaOffset()) {
committedKafkaOffset = ongoingTransactionInfo.getLastWrittenKafkaOffset();
// Update committedKafkaOffset that tracks the last committed kafka offset locally.
if (ongoingTransactionInfo != null && committedKafkaOffset < ongoingTransactionInfo.getExpectedKafkaOffset()) {
committedKafkaOffset = ongoingTransactionInfo.getExpectedKafkaOffset();
}
syncKafkaOffsetWithLeader(message);
cleanupOngoingTransaction();
@@ -215,12 +215,22 @@ public class ConnectTransactionParticipant implements TransactionParticipant {
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(),
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());
} else if (record != null && record.kafkaOffset() < ongoingTransactionInfo.getExpectedKafkaOffset()) {
LOG.warn(String.format("Received a kafka record with offset %s below the next expected kafka offset %s for partition %s, "
+ "no action will be taken but this record will be ignored since its already written",
record.kafkaOffset(),
ongoingTransactionInfo.getExpectedKafkaOffset(),
partition));
}
buffer.poll();
@@ -250,13 +260,24 @@ public class ConnectTransactionParticipant implements TransactionParticipant {
// Recover kafka committed offsets, treating the commit offset from the coordinator
// as the source of truth
if (coordinatorCommittedKafkaOffset != null && coordinatorCommittedKafkaOffset >= 0) {
// Debug only messages
if (coordinatorCommittedKafkaOffset != committedKafkaOffset) {
LOG.warn(String.format("Recovering the kafka offset for partition %s to offset %s instead of local offset %s",
partition.partition(), coordinatorCommittedKafkaOffset, committedKafkaOffset));
context.offset(partition, coordinatorCommittedKafkaOffset);
LOG.warn(String.format("The coordinator offset for kafka partition %s is %d while the locally committed offset is %d, "
+ "hence resetting the local committed offset to the coordinator provided one to ensure consistency",
partition,
coordinatorCommittedKafkaOffset,
committedKafkaOffset));
}
committedKafkaOffset = coordinatorCommittedKafkaOffset;
return;
}
} else {
LOG.warn(String.format("The coordinator offset for kafka partition %s is not present while the locally committed offset is %d, "
+ "hence resetting the local committed offset to 0 to avoid data loss",
partition,
committedKafkaOffset));
}
// If the coordinator does not have a committed offset for this partition, reset to zero offset.
committedKafkaOffset = 0;
}
}

View File

@@ -29,13 +29,13 @@ public class TransactionInfo<T> {
private final String commitTime;
private final ConnectWriter<T> writer;
private long lastWrittenKafkaOffset;
private long expectedKafkaOffset;
private boolean commitInitiated;
public TransactionInfo(String commitTime, ConnectWriter<T> writer) {
this.commitTime = commitTime;
this.writer = writer;
this.lastWrittenKafkaOffset = 0;
this.expectedKafkaOffset = 0;
this.commitInitiated = false;
}
@@ -47,16 +47,16 @@ public class TransactionInfo<T> {
return writer;
}
public long getLastWrittenKafkaOffset() {
return lastWrittenKafkaOffset;
public long getExpectedKafkaOffset() {
return expectedKafkaOffset;
}
public boolean isCommitInitiated() {
return commitInitiated;
}
public void setLastWrittenKafkaOffset(long lastWrittenKafkaOffset) {
this.lastWrittenKafkaOffset = lastWrittenKafkaOffset;
public void setExpectedKafkaOffset(long expectedKafkaOffset) {
this.expectedKafkaOffset = expectedKafkaOffset;
}
public void commitInitiated() {