Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,9 @@ public RangeState getOverLappingState(RangeContainer rangeContainer) {
// ZEROED [10, 20] Actual [0, 10]
if (actualMinOffset == 0)
return RangeState.ZERO;
// PREVIOUS
if (actualMaxOffset < minOffset)
return RangeState.PREVIOUS;
// ERROR [10, 20] Actual [8, 19]
return RangeState.ERROR;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -1,15 +1,15 @@
package com.clickhouse.kafka.connect.sink.kafka;

public enum RangeState {

ZERO(0), //This is for when it seems like the topic has been deleted/recreated
SAME(1),
PREFIX(2),
SUFFIX(3),
CONTAINS(4),
OVER_LAPPING(5),
NEW(6),
ERROR(7);
ERROR(7),
PREVIOUS(8);


private int rangeState;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ public void doLogic(List<Record> records) throws IOException, ExecutionException
doInsert(rightRecords, rightRangeContainer);
stateProvider.setStateRecord(new StateRecord(topic, partition, rightRangeContainer.getMaxOffset(), rightRangeContainer.getMinOffset(), State.AFTER_PROCESSING));
break;
case ERROR:
default: //case ERROR:
throw new RuntimeException(String.format("State MISMATCH given [%s] records for topic: [%s], partition: [%s], minOffset: [%s], maxOffset: [%s], expectedMinOffset: [%s], expectedMaxOffset: [%s]",
records.size(), topic, partition, rangeContainer.getMinOffset(), rangeContainer.getMaxOffset(), stateRecord.getMinOffset(), stateRecord.getMaxOffset()));
}
Expand Down Expand Up @@ -211,7 +211,11 @@ public void doLogic(List<Record> records) throws IOException, ExecutionException
doInsert(rightRecords, rightRangeContainer);
stateProvider.setStateRecord(new StateRecord(topic, partition, rightRangeContainer.getMaxOffset(), rightRangeContainer.getMinOffset(), State.AFTER_PROCESSING));
break;
case ERROR:
case PREVIOUS:
LOGGER.warn("Batch already processed - skipping [{}] records for topic: [{}], partition: [{}], minOffset: [{}], maxOffset: [{}], storedMinOffset: [{}], storedMaxOffset: [{}]",
records.size(), topic, partition, rangeContainer.getMinOffset(), rangeContainer.getMaxOffset(), stateRecord.getMinOffset(), stateRecord.getMaxOffset());
break;
default: //case ERROR:
throw new RuntimeException(String.format("State MISMATCH given [%s] records for topic: [%s], partition: [%s], minOffset: [%s], maxOffset: [%s], expectedMinOffset: [%s], expectedMaxOffset: [%s]",
records.size(), topic, partition, rangeContainer.getMinOffset(), rangeContainer.getMaxOffset(), stateRecord.getMinOffset(), stateRecord.getMaxOffset()));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -244,4 +244,17 @@ public void ProcessPartialOverlappingAfterProcessingTest() throws IOException, E
assertEquals(records.size(), dbWriter.recordsInserted());
}

@Test
@DisplayName("ProcessOldRecordsTest")
public void ProcessOldRecordsTest() throws IOException, ExecutionException, InterruptedException {
List<Record> records = createRecords("test", 1);
List<Record> recordsHead = records.subList(1, 2);
StateProvider stateProvider = new InMemoryState();
stateProvider.setStateRecord(new StateRecord("test", 1, 5000, 4000, State.AFTER_PROCESSING));
DBWriter dbWriter = new InMemoryDBWriter();
Processing processing = new Processing(stateProvider, dbWriter);
processing.doLogic(recordsHead);
assertEquals(0, dbWriter.recordsInserted());
}

}
Loading