Skip to content
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
20 changes: 17 additions & 3 deletions src/main/java/de/azapps/kafkabackup/sink/BackupSinkTask.java
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTask;
import org.apache.kafka.connect.sink.SinkTaskContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -23,6 +22,7 @@
import java.util.*;

public class BackupSinkTask extends SinkTask {
private static final Long DEFAULT_OFFSET = Long.valueOf(-1L);
private static final Logger log = LoggerFactory.getLogger(BackupSinkTask.class);
private Path targetDir;
private Map<TopicPartition, PartitionWriter> partitionWriters = new HashMap<>();
Expand All @@ -33,6 +33,7 @@ public class BackupSinkTask extends SinkTask {
private Map<TopicPartition, Long> currentOffsets = new HashMap<>();
private EndOffsetReader endOffsetReader;
private java.util.function.Consumer<Integer> exitFunction;
private Set<TopicPartition> topicPartitionsWithDefaultOffset = new HashSet();

@Override
public String version() {
Expand Down Expand Up @@ -90,9 +91,22 @@ private void terminateIfCompleted() {
boolean terminate = true;
for (Map.Entry<TopicPartition, Long> partitionOffset : endOffsets.entrySet()) {
Long endOffset = partitionOffset.getValue();
Long currentOffset = currentOffsets.getOrDefault(partitionOffset.getKey(), -1L);
Long currentOffset = currentOffsets.getOrDefault(partitionOffset.getKey(), DEFAULT_OFFSET);

if (currentOffset < endOffset - 1) {
final TopicPartition topicPartition = partitionOffset.getKey();
log.debug("check to terminate: partition {} current offset {} end offset {}", topicPartition, currentOffset, endOffset);
boolean topicPartitionIsEmpty = false;

if (DEFAULT_OFFSET.equals(currentOffset)) {
if (topicPartitionsWithDefaultOffset.contains(topicPartition)) {
topicPartitionIsEmpty = true;
log.info("topic partition is empty {}", topicPartition);
} else {
topicPartitionsWithDefaultOffset.add(topicPartition);
}
}

if (!topicPartitionIsEmpty && currentOffset < endOffset - 1) {
return;
}
}
Expand Down