Skip to content

Commit b41a27f

Browse files
authored
Purge events tied to closed partitions (#259)
In some cases, after a rebalance of partitions and depending on the late arrival of acks from HEC, it's needed to remove all references to events tied to the closed partitions. This purge has to be made in buffered events, failed events, offsets, and topic/partition records. This is considered safe because those events will be picked up by the task that opens the partitions closed in the original task.
1 parent c67df74 commit b41a27f

File tree

4 files changed

+119
-20
lines changed

4 files changed

+119
-20
lines changed

src/main/java/com/splunk/kafka/connect/KafkaRecordTracker.java

+78-19
Original file line numberDiff line numberDiff line change
@@ -38,39 +38,42 @@ final class KafkaRecordTracker {
3838
private AtomicLong total;
3939
private ConcurrentLinkedQueue<EventBatch> failed;
4040
private volatile Map<TopicPartition, OffsetAndMetadata> offsets;
41+
private Collection<TopicPartition> partitions;
4142

4243
public KafkaRecordTracker() {
4344
all = new ConcurrentHashMap<>();
4445
failed = new ConcurrentLinkedQueue<>();
4546
total = new AtomicLong();
4647
offsets = new HashMap<>();
48+
partitions = new ArrayList<TopicPartition>();
4749
}
4850

51+
/**
52+
* Remove acked events and update the corresponding offsets finding the
53+
* lowest consecutive HEC-commited offsets.
54+
*
55+
* @param batches the acked event batches
56+
*/
4957
public void removeAckedEventBatches(final List<EventBatch> batches) {
50-
for (final EventBatch batch: batches) {
51-
//log.debug("Processing batch {}", batch.getUUID());
52-
removeAckedEventBatch(batch);
53-
}
54-
}
55-
56-
public void removeAckedEventBatch(final EventBatch batch) {
57-
final List<Event> events = batch.getEvents();
58-
final Event event = events.get(0);
59-
if (event.getTied() instanceof SinkRecord) {
60-
final SinkRecord record = (SinkRecord) event.getTied();
61-
TopicPartition tp = new TopicPartition(record.topic(), record.kafkaPartition());
62-
//log.debug("Processing topic {} partition {}", record.topic(), record.kafkaPartition());
58+
log.debug("received acked event batches={}", batches);
59+
/* Loop all *assigned* partitions to find the lowest consecutive
60+
* HEC-commited offsets. A batch could contain events coming from a
61+
* variety of topic/partitions, and scanning those events coulb be
62+
* expensive.
63+
* Note that if some events are tied to an unassigned partition those
64+
* offsets won't be able to be commited.
65+
*/
66+
for (TopicPartition tp : partitions) {
6367
ConcurrentNavigableMap<Long, EventBatch> tpRecords = all.get(tp);
6468
if (tpRecords == null) {
65-
log.error("KafkaRecordTracker removing a batch in an unknown partition {} {} {}", record.topic(), record.kafkaPartition(), record.kafkaOffset());
66-
return;
69+
continue; // nothing to remove in this case
6770
}
6871
long offset = -1;
6972
Iterator<Map.Entry<Long, EventBatch>> iter = tpRecords.entrySet().iterator();
7073
for (; iter.hasNext();) {
7174
Map.Entry<Long, EventBatch> e = iter.next();
7275
if (e.getValue().isCommitted()) {
73-
//log.debug("processing offset {}", e.getKey());
76+
log.debug("processing offset {}", e.getKey());
7477
offset = e.getKey();
7578
iter.remove();
7679
total.decrementAndGet();
@@ -107,21 +110,77 @@ public Collection<EventBatch> getAndRemoveFailedRecords() {
107110
Collection<EventBatch> records = new ArrayList<>();
108111
while (!failed.isEmpty()) {
109112
final EventBatch batch = failed.poll();
113+
/* Don't return null batches. */
110114
if (batch != null) {
115+
/* Purge events from closed partitions because it won't be
116+
* possible to commit their offsets. */
117+
batch.getEvents().removeIf(e -> !partitions.contains(getPartitionFromEvent(e)));
111118
records.add(batch);
112119
}
113120
}
114121
return records;
115122
}
116123

117-
// Loop through all SinkRecords for all topic partitions to
118-
// find all lowest consecutive committed offsets, calculate
119-
// the topic/partition offsets and then remove them
124+
/**
125+
* Return offsets computed when event batches are acked.
126+
*
127+
* @return map of topic/partition to offset/metadata
128+
*/
120129
public Map<TopicPartition, OffsetAndMetadata> computeOffsets() {
121130
return offsets;
122131
}
123132

124133
public long totalEvents() {
125134
return total.get();
126135
}
136+
137+
public void open(Collection<TopicPartition> partitions) {
138+
this.partitions.addAll(partitions);
139+
log.debug("open partitions={} so currently assigned partitions={}",
140+
partitions, this.partitions);
141+
}
142+
143+
public void close(Collection<TopicPartition> partitions) {
144+
this.partitions.removeAll(partitions);
145+
log.debug("close partitions={} so currently assigned partitions={}",
146+
partitions, this.partitions);
147+
cleanupAfterClosedPartitions(partitions);
148+
}
149+
150+
private TopicPartition getPartitionFromEvent(Event event) {
151+
if (event.getTied() instanceof SinkRecord) {
152+
final SinkRecord r = (SinkRecord) event.getTied();
153+
return new TopicPartition(r.topic(), r.kafkaPartition());
154+
} else return null;
155+
}
156+
157+
/**
158+
* Clean up and purge all things related to a partition that's closed (i.e.
159+
* became unassigned) to this task and reported via SinkTask.close(). This
160+
* avoids race conditions related to late received acks after a partition
161+
* rebalance.
162+
*
163+
* @param partitions partition closed and now unassigned for this task
164+
*/
165+
public void cleanupAfterClosedPartitions(Collection<TopicPartition> partitions)
166+
{
167+
/* Purge offsets. */
168+
offsets.keySet().removeAll(partitions);
169+
log.warn("purge offsets for closed partitions={} leaving offsets={}",
170+
partitions, offsets);
171+
172+
/* Count and purge outstanding event topic/partition records. */
173+
long countOfEventsToRemove = partitions.stream()
174+
.map(tp -> all.get(tp)) // get unassigned topic/partition records
175+
.filter(Objects::nonNull) // filter out null values
176+
.map(tpr -> tpr.size()) // get number of tp records
177+
.mapToInt(Integer::intValue) // map to int
178+
.sum();
179+
if (countOfEventsToRemove > 0) {
180+
log.warn("purge events={} from closed partitions={}",
181+
countOfEventsToRemove, partitions);
182+
all.keySet().removeAll(partitions);
183+
total.addAndGet(-1L * countOfEventsToRemove);
184+
}
185+
}
127186
}

src/main/java/com/splunk/kafka/connect/SplunkSinkTask.java

+15
Original file line numberDiff line numberDiff line change
@@ -325,6 +325,21 @@ private EventBatch createRawEventBatch(final TopicPartition tp) {
325325
.build();
326326
}
327327

328+
@Override
329+
public void open(Collection<TopicPartition> partitions) {
330+
tracker.open(partitions);
331+
}
332+
333+
@Override
334+
public void close(Collection<TopicPartition> partitions) {
335+
/* Purge buffered events tied to closed partitions because this task
336+
* won't be able to commit their offsets. */
337+
bufferedRecords.removeIf(r -> partitions.contains(
338+
new TopicPartition(r.topic(), r.kafkaPartition())));
339+
/* Tell tracker about now closed partitions so to clean up. */
340+
tracker.close(partitions);
341+
}
342+
328343
@Override
329344
public Map<TopicPartition, OffsetAndMetadata> preCommit(Map<TopicPartition, OffsetAndMetadata> meta) {
330345
// tell Kafka Connect framework what are offsets we can safely commit to Kafka now

src/test/java/com/splunk/kafka/connect/KafkaRecordTrackerTest.java

+9-1
Original file line numberDiff line numberDiff line change
@@ -33,8 +33,9 @@ public class KafkaRecordTrackerTest {
3333
public void addFailedEventBatch() {
3434
EventBatch batch = UnitUtil.createBatch();
3535
batch.fail();
36-
36+
batch.getEvents().get(0).setTied(createSinkRecord(1));
3737
KafkaRecordTracker tracker = new KafkaRecordTracker();
38+
tracker.open(createTopicPartitionList());
3839
tracker.addFailedEventBatch(batch);
3940
Collection<EventBatch> failed = tracker.getAndRemoveFailedRecords();
4041
Assert.assertEquals(1, failed.size());
@@ -55,6 +56,7 @@ public void addEventBatch() {
5556
EventBatch batch = UnitUtil.createBatch();
5657
batch.getEvents().get(0).setTied(createSinkRecord(i));
5758
batches.add(batch);
59+
tracker.open(createTopicPartitionList());
5860
tracker.addEventBatch(batch);
5961
}
6062
Map<TopicPartition, OffsetAndMetadata> offsets = tracker.computeOffsets();
@@ -96,4 +98,10 @@ public void addEventBatchWithNonSinkRecord() {
9698
private SinkRecord createSinkRecord(long offset) {
9799
return new SinkRecord("t", 1, null, null, null, "ni, hao", offset);
98100
}
101+
102+
private List<TopicPartition> createTopicPartitionList() {
103+
ArrayList<TopicPartition> tps = new ArrayList<>();
104+
tps.add(new TopicPartition("t", 1));
105+
return tps;
106+
}
99107
}

src/test/java/com/splunk/kafka/connect/SplunkSinkTaskTest.java

+17
Original file line numberDiff line numberDiff line change
@@ -78,16 +78,21 @@ public void putWithoutMaxBatchAligned() {
7878

7979
SplunkSinkTask task = new SplunkSinkTask();
8080
HecMock hec = new HecMock(task);
81+
TopicPartition tp = new TopicPartition(uu.configProfile.getTopics(), 1);
82+
List<TopicPartition> partitions = new ArrayList<>();
83+
partitions.add(tp);
8184
// success
8285
hec.setSendReturnResult(HecMock.success);
8386
task.setHec(hec);
8487
task.start(config);
88+
task.open(partitions);
8589
task.put(createSinkRecords(120));
8690
Assert.assertEquals(2, hec.getBatches().size());
8791
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
8892
offsets.put(new TopicPartition(uu.configProfile.getTopics(), 1), new OffsetAndMetadata(120));
8993
Assert.assertEquals(offsets, task.preCommit(new HashMap<>()));
9094
Assert.assertTrue(task.getTracker().getAndRemoveFailedRecords().isEmpty());
95+
task.close(partitions);
9196
task.stop();
9297
}
9398

@@ -105,6 +110,7 @@ public void putWithFailure() {
105110
hec.setSendReturnResult(HecMock.failure);
106111
task.setHec(hec);
107112
task.start(config);
113+
task.open(createTopicPartitionList());
108114
task.put(createSinkRecords(1000));
109115
Assert.assertEquals(10, hec.getBatches().size());
110116
Assert.assertTrue(task.getTracker().computeOffsets().isEmpty());
@@ -266,10 +272,14 @@ private void putWithSuccess(boolean raw, boolean withMeta) {
266272

267273
SplunkSinkTask task = new SplunkSinkTask();
268274
HecMock hec = new HecMock(task);
275+
TopicPartition tp = new TopicPartition(uu.configProfile.getTopics(), 1);
276+
List<TopicPartition> partitions = new ArrayList<>();
277+
partitions.add(tp);
269278
// success
270279
hec.setSendReturnResult(HecMock.success);
271280
task.setHec(hec);
272281
task.start(config);
282+
task.open(partitions);
273283
task.put(createSinkRecords(total));
274284
Assert.assertEquals(10, hec.getBatches().size());
275285
if (raw && withMeta) {
@@ -303,6 +313,7 @@ private void putWithSuccess(boolean raw, boolean withMeta) {
303313
offsets.put(new TopicPartition(uu.configProfile.getTopics(), 1), new OffsetAndMetadata(1000));
304314
Assert.assertEquals(offsets, task.preCommit(new HashMap<>()));
305315
Assert.assertTrue(task.getTracker().getAndRemoveFailedRecords().isEmpty());
316+
task.close(partitions);
306317
task.stop();
307318
}
308319

@@ -329,4 +340,10 @@ private Collection<SinkRecord> createNullSinkRecord() {
329340
records.add(rec);
330341
return records;
331342
}
343+
344+
private List<TopicPartition> createTopicPartitionList() {
345+
ArrayList<TopicPartition> tps = new ArrayList<>();
346+
tps.add(new TopicPartition("mytopic", 1));
347+
return tps;
348+
}
332349
}

0 commit comments

Comments
 (0)