Skip to content

Commit 80ab55c

Browse files
authoredJan 10, 2025··
SKYEDEN-3020 | consumers retransmission improvements (#1941)
* SKYEDEN-3020 | commiting offsets change on retransmission * SKYEDEN-3020 | prevent overriding commited offset with lower offset * SKYEDEN-3020 | tests + new approach * SKYEDEN-3020 | add comment * SKYEDEN-3020 | resolve conflicts * SKYEDEN-3020 | refactor
1 parent f551c5b commit 80ab55c

File tree

16 files changed

+121
-62
lines changed

16 files changed

+121
-62
lines changed
 

‎hermes-common/src/main/java/pl/allegro/tech/hermes/common/kafka/offset/PartitionOffsets.java

+4
Original file line numberDiff line numberDiff line change
@@ -22,4 +22,8 @@ public PartitionOffsets addAll(PartitionOffsets offsets) {
2222
public Iterator<PartitionOffset> iterator() {
2323
return offsets.iterator();
2424
}
25+
26+
public boolean isEmpty() {
27+
return offsets.isEmpty();
28+
}
2529
}

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java

+4-4
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@
2222
import pl.allegro.tech.hermes.api.BatchSubscriptionPolicy;
2323
import pl.allegro.tech.hermes.api.Subscription;
2424
import pl.allegro.tech.hermes.api.Topic;
25-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
25+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
2626
import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper;
2727
import pl.allegro.tech.hermes.common.metric.MetricsFacade;
2828
import pl.allegro.tech.hermes.consumers.consumer.batch.MessageBatch;
@@ -239,11 +239,11 @@ public void commit(Set<SubscriptionPartitionOffset> offsetsToCommit) {
239239
}
240240

241241
@Override
242-
public boolean moveOffset(PartitionOffset partitionOffset) {
242+
public PartitionOffsets moveOffset(PartitionOffsets partitionOffsets) {
243243
if (receiver != null) {
244-
return receiver.moveOffset(partitionOffset);
244+
return receiver.moveOffset(partitionOffsets);
245245
}
246-
return false;
246+
return new PartitionOffsets();
247247
}
248248

249249
@Override

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Consumer.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,7 @@
33
import java.util.Set;
44
import pl.allegro.tech.hermes.api.Subscription;
55
import pl.allegro.tech.hermes.api.Topic;
6-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
6+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
77
import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset;
88

99
public interface Consumer {
@@ -26,7 +26,7 @@ public interface Consumer {
2626

2727
void commit(Set<SubscriptionPartitionOffset> offsets);
2828

29-
boolean moveOffset(PartitionOffset subscriptionPartitionOffset);
29+
PartitionOffsets moveOffset(PartitionOffsets subscriptionPartitionOffsets);
3030

3131
Subscription getSubscription();
3232

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SerialConsumer.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -11,7 +11,7 @@
1111
import org.slf4j.LoggerFactory;
1212
import pl.allegro.tech.hermes.api.Subscription;
1313
import pl.allegro.tech.hermes.api.Topic;
14-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
14+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
1515
import pl.allegro.tech.hermes.common.metric.MetricsFacade;
1616
import pl.allegro.tech.hermes.consumers.CommonConsumerParameters;
1717
import pl.allegro.tech.hermes.consumers.consumer.converter.MessageConverterResolver;
@@ -262,8 +262,8 @@ public void commit(Set<SubscriptionPartitionOffset> offsets) {
262262
}
263263

264264
@Override
265-
public boolean moveOffset(PartitionOffset offset) {
266-
return messageReceiver.moveOffset(offset);
265+
public PartitionOffsets moveOffset(PartitionOffsets offsets) {
266+
return messageReceiver.moveOffset(offsets);
267267
}
268268

269269
@Override

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/batch/MessageBatchReceiver.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,7 @@
1616
import org.slf4j.LoggerFactory;
1717
import pl.allegro.tech.hermes.api.Subscription;
1818
import pl.allegro.tech.hermes.api.Topic;
19-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
19+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
2020
import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper;
2121
import pl.allegro.tech.hermes.common.message.wrapper.UnsupportedContentTypeException;
2222
import pl.allegro.tech.hermes.consumers.consumer.Message;
@@ -180,8 +180,8 @@ public void commit(Set<SubscriptionPartitionOffset> offsets) {
180180
receiver.commit(offsets);
181181
}
182182

183-
public boolean moveOffset(PartitionOffset offset) {
184-
return receiver.moveOffset(offset);
183+
public PartitionOffsets moveOffset(PartitionOffsets offsets) {
184+
return receiver.moveOffset(offsets);
185185
}
186186

187187
public Set<Integer> getAssignedPartitions() {

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/ConsumerPartitionAssignmentState.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ public void assign(SubscriptionName name, Collection<Integer> partitions) {
3535
}));
3636
}
3737

38-
private void incrementTerm(SubscriptionName name) {
38+
public void incrementTerm(SubscriptionName name) {
3939
terms.compute(name, ((subscriptionName, term) -> term == null ? 0L : term + 1L));
4040
}
4141

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/kafka/broker/KafkaConsumerOffsetMover.java

+52-2
Original file line numberDiff line numberDiff line change
@@ -1,25 +1,55 @@
11
package pl.allegro.tech.hermes.consumers.consumer.offset.kafka.broker;
22

3+
import java.util.LinkedHashMap;
4+
import java.util.Map;
35
import org.apache.kafka.clients.consumer.KafkaConsumer;
6+
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
47
import org.apache.kafka.common.TopicPartition;
58
import org.slf4j.Logger;
69
import org.slf4j.LoggerFactory;
710
import pl.allegro.tech.hermes.api.SubscriptionName;
811
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
12+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
13+
import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState;
914

1015
public class KafkaConsumerOffsetMover {
1116

1217
private static final Logger logger = LoggerFactory.getLogger(KafkaConsumerOffsetMover.class);
1318

1419
private final SubscriptionName subscriptionName;
1520
private KafkaConsumer consumer;
21+
private ConsumerPartitionAssignmentState partitionAssignmentState;
1622

17-
public KafkaConsumerOffsetMover(SubscriptionName subscriptionName, KafkaConsumer consumer) {
23+
public KafkaConsumerOffsetMover(
24+
SubscriptionName subscriptionName,
25+
KafkaConsumer consumer,
26+
ConsumerPartitionAssignmentState partitionAssignmentState) {
1827
this.subscriptionName = subscriptionName;
1928
this.consumer = consumer;
29+
this.partitionAssignmentState = partitionAssignmentState;
2030
}
2131

22-
public boolean move(PartitionOffset offset) {
32+
public PartitionOffsets move(PartitionOffsets offsets) {
33+
PartitionOffsets movedOffsets = new PartitionOffsets();
34+
35+
for (PartitionOffset offset : offsets) {
36+
if (move(offset)) {
37+
movedOffsets.add(offset);
38+
}
39+
}
40+
41+
commit(movedOffsets);
42+
43+
if (!movedOffsets.isEmpty()) {
44+
// Incrementing assignment term ensures that currently committed offsets won't be overwritten
45+
// by the events from the past which are concurrently processed by the consumer
46+
partitionAssignmentState.incrementTerm(subscriptionName);
47+
}
48+
49+
return movedOffsets;
50+
}
51+
52+
private boolean move(PartitionOffset offset) {
2353
try {
2454
TopicPartition tp = new TopicPartition(offset.getTopic().asString(), offset.getPartition());
2555
if (consumer.assignment().contains(tp)) {
@@ -46,4 +76,24 @@ public boolean move(PartitionOffset offset) {
4676
return false;
4777
}
4878
}
79+
80+
private void commit(PartitionOffsets partitionOffsets) {
81+
try {
82+
Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = new LinkedHashMap<>();
83+
for (PartitionOffset partitionOffset : partitionOffsets) {
84+
offsetsToCommit.put(
85+
new TopicPartition(
86+
partitionOffset.getTopic().asString(), partitionOffset.getPartition()),
87+
new OffsetAndMetadata(partitionOffset.getOffset()));
88+
}
89+
if (!offsetsToCommit.isEmpty()) {
90+
consumer.commitSync(offsetsToCommit);
91+
}
92+
} catch (Exception e) {
93+
logger.error(
94+
"Failed to commit offsets while trying to move them for subscription {}",
95+
subscriptionName,
96+
e);
97+
}
98+
}
4999
}

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/MessageReceiver.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,7 @@
33
import java.util.Optional;
44
import java.util.Set;
55
import pl.allegro.tech.hermes.api.Subscription;
6-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
6+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
77
import pl.allegro.tech.hermes.consumers.consumer.Message;
88
import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset;
99

@@ -33,7 +33,7 @@ default void update(Subscription newSubscription) {}
3333

3434
void commit(Set<SubscriptionPartitionOffset> offsets);
3535

36-
boolean moveOffset(PartitionOffset offset);
36+
PartitionOffsets moveOffset(PartitionOffsets offsets);
3737

3838
Set<Integer> getAssignedPartitions();
3939
}

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ThrottlingMessageReceiver.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@
44
import java.util.Set;
55
import pl.allegro.tech.hermes.api.Subscription;
66
import pl.allegro.tech.hermes.api.SubscriptionName;
7-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
7+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
88
import pl.allegro.tech.hermes.common.metric.MetricsFacade;
99
import pl.allegro.tech.hermes.consumers.consumer.Message;
1010
import pl.allegro.tech.hermes.consumers.consumer.idletime.IdleTimeCalculator;
@@ -53,8 +53,8 @@ public void commit(Set<SubscriptionPartitionOffset> offsets) {
5353
}
5454

5555
@Override
56-
public boolean moveOffset(PartitionOffset offset) {
57-
return receiver.moveOffset(offset);
56+
public PartitionOffsets moveOffset(PartitionOffsets offsets) {
57+
return receiver.moveOffset(offsets);
5858
}
5959

6060
@Override

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/UninitializedMessageReceiver.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,7 @@
22

33
import java.util.Optional;
44
import java.util.Set;
5-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
5+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
66
import pl.allegro.tech.hermes.consumers.consumer.Message;
77
import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset;
88

@@ -18,7 +18,7 @@ public void commit(Set<SubscriptionPartitionOffset> offsets) {
1818
}
1919

2020
@Override
21-
public boolean moveOffset(PartitionOffset offset) {
21+
public PartitionOffsets moveOffset(PartitionOffsets offsets) {
2222
throw new ConsumerNotInitializedException();
2323
}
2424

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/FilteringMessageReceiver.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@
44
import java.util.Optional;
55
import java.util.Set;
66
import pl.allegro.tech.hermes.api.Subscription;
7-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
7+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
88
import pl.allegro.tech.hermes.consumers.consumer.Message;
99
import pl.allegro.tech.hermes.consumers.consumer.filtering.FilteredMessageHandler;
1010
import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset;
@@ -65,8 +65,8 @@ public void commit(Set<SubscriptionPartitionOffset> offsets) {
6565
}
6666

6767
@Override
68-
public boolean moveOffset(PartitionOffset offset) {
69-
return receiver.moveOffset(offset);
68+
public PartitionOffsets moveOffset(PartitionOffsets offsets) {
69+
return receiver.moveOffset(offsets);
7070
}
7171

7272
@Override

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaSingleThreadedMessageReceiver.java

+7-4
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@
2525
import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper;
2626
import pl.allegro.tech.hermes.common.kafka.KafkaTopic;
2727
import pl.allegro.tech.hermes.common.kafka.KafkaTopics;
28-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset;
28+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets;
2929
import pl.allegro.tech.hermes.common.metric.MetricsFacade;
3030
import pl.allegro.tech.hermes.consumers.consumer.Message;
3131
import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder;
@@ -74,7 +74,9 @@ public KafkaSingleThreadedMessageReceiver(
7474
this.partitionAssignmentState = partitionAssignmentState;
7575
this.consumer = consumer;
7676
this.readQueue = new ArrayBlockingQueue<>(readQueueCapacity);
77-
this.offsetMover = new KafkaConsumerOffsetMover(subscription.getQualifiedName(), consumer);
77+
this.offsetMover =
78+
new KafkaConsumerOffsetMover(
79+
subscription.getQualifiedName(), consumer, partitionAssignmentState);
7880
Map<String, KafkaTopic> topics =
7981
getKafkaTopics(topic, kafkaNamesMapper).stream()
8082
.collect(Collectors.toMap(t -> t.name().asString(), Function.identity()));
@@ -195,6 +197,7 @@ public void commit(Set<SubscriptionPartitionOffset> offsets) {
195197

196198
private Map<TopicPartition, OffsetAndMetadata> createOffset(
197199
Set<SubscriptionPartitionOffset> partitionOffsets) {
200+
198201
Map<TopicPartition, OffsetAndMetadata> offsetsData = new LinkedHashMap<>();
199202
for (SubscriptionPartitionOffset partitionOffset : partitionOffsets) {
200203
TopicPartition topicAndPartition =
@@ -223,8 +226,8 @@ private Map<TopicPartition, OffsetAndMetadata> createOffset(
223226
}
224227

225228
@Override
226-
public boolean moveOffset(PartitionOffset offset) {
227-
return offsetMover.move(offset);
229+
public PartitionOffsets moveOffset(PartitionOffsets offsets) {
230+
return offsetMover.move(offsets);
228231
}
229232

230233
public Set<Integer> getAssignedPartitions() {

‎hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/Retransmitter.java

+13-27
Original file line numberDiff line numberDiff line change
@@ -33,36 +33,22 @@ public void reloadOffsets(SubscriptionName subscriptionName, Consumer consumer)
3333
brokersClusterName,
3434
consumer.getAssignedPartitions());
3535

36-
for (PartitionOffset partitionOffset : offsets) {
37-
if (moveOffset(subscriptionName, consumer, partitionOffset)) {
38-
subscriptionOffsetChangeIndicator.removeOffset(
39-
subscriptionName.getTopicName(),
40-
subscriptionName.getName(),
41-
brokersClusterName,
42-
partitionOffset.getTopic(),
43-
partitionOffset.getPartition());
44-
logger.info(
45-
"Removed offset indicator for subscription={} and partition={}",
46-
subscriptionName,
47-
partitionOffset.getPartition());
48-
}
36+
PartitionOffsets movedOffsets = consumer.moveOffset(offsets);
37+
38+
for (PartitionOffset partitionOffset : movedOffsets) {
39+
subscriptionOffsetChangeIndicator.removeOffset(
40+
subscriptionName.getTopicName(),
41+
subscriptionName.getName(),
42+
brokersClusterName,
43+
partitionOffset.getTopic(),
44+
partitionOffset.getPartition());
45+
logger.info(
46+
"Removed offset indicator for subscription={} and partition={}",
47+
subscriptionName,
48+
partitionOffset.getPartition());
4949
}
5050
} catch (Exception ex) {
5151
throw new RetransmissionException(ex);
5252
}
5353
}
54-
55-
private boolean moveOffset(
56-
SubscriptionName subscriptionName, Consumer consumer, PartitionOffset partitionOffset) {
57-
try {
58-
return consumer.moveOffset(partitionOffset);
59-
} catch (IllegalStateException ex) {
60-
logger.warn(
61-
"Cannot move offset for subscription={} and partition={} , possibly owned by different node",
62-
subscriptionName,
63-
partitionOffset.getPartition(),
64-
ex);
65-
return false;
66-
}
67-
}
6854
}

‎hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerStub.groovy

+3-3
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,7 @@ package pl.allegro.tech.hermes.consumers.supervisor.process
22

33
import pl.allegro.tech.hermes.api.Subscription
44
import pl.allegro.tech.hermes.api.Topic
5-
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset
5+
import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffsets
66
import pl.allegro.tech.hermes.consumers.consumer.Consumer
77
import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset
88

@@ -78,8 +78,8 @@ class ConsumerStub implements Consumer {
7878
}
7979

8080
@Override
81-
boolean moveOffset(PartitionOffset partitionOffset) {
82-
return true
81+
PartitionOffsets moveOffset(PartitionOffsets partitionOffset) {
82+
return partitionOffset
8383
}
8484

8585
boolean getInitialized() {

‎hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -189,7 +189,7 @@ public void waitUntilConsumerCommitsOffset(String topicQualifiedName, String sub
189189
});
190190
}
191191

192-
private long calculateCommittedMessages(String topicQualifiedName, String subscription) {
192+
public long calculateCommittedMessages(String topicQualifiedName, String subscription) {
193193
AtomicLong messagesCommittedCount = new AtomicLong(0);
194194
List<ConsumerGroup> consumerGroups =
195195
getConsumerGroupsDescription(topicQualifiedName, subscription)

‎integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java

+18-2
Original file line numberDiff line numberDiff line change
@@ -75,6 +75,9 @@ public void shouldMoveOffsetNearGivenTimestamp(boolean suspendedSubscription)
7575
publishAndConsumeMessages(messages2, topic, subscriber);
7676
hermes.api().waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName());
7777

78+
long commitedMessages =
79+
hermes.api().calculateCommittedMessages(topic.getQualifiedName(), subscription.getName());
80+
7881
if (suspendedSubscription) {
7982
hermes.api().suspendSubscription(topic, subscription.getName());
8083
hermes.api().waitUntilSubscriptionSuspended(topic.getQualifiedName(), subscription.getName());
@@ -87,14 +90,27 @@ public void shouldMoveOffsetNearGivenTimestamp(boolean suspendedSubscription)
8790
.retransmit(
8891
topic.getQualifiedName(), subscription.getName(), retransmissionDate, false);
8992

93+
// then
94+
response.expectStatus().isOk();
95+
// Check if Kafka committed offsets were moved on retransmission
96+
assertThat(
97+
hermes
98+
.api()
99+
.calculateCommittedMessages(topic.getQualifiedName(), subscription.getName()))
100+
.isLessThan(commitedMessages);
101+
90102
if (suspendedSubscription) {
91103
hermes.api().activateSubscription(topic, subscription.getName());
92104
hermes.api().waitUntilSubscriptionActivated(topic.getQualifiedName(), subscription.getName());
93105
}
94106

95-
// then
96-
response.expectStatus().isOk();
97107
messages2.forEach(subscriber::waitUntilReceived);
108+
hermes.api().waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName());
109+
assertThat(
110+
hermes
111+
.api()
112+
.calculateCommittedMessages(topic.getQualifiedName(), subscription.getName()))
113+
.isEqualTo(commitedMessages);
98114
}
99115

100116
@Test

0 commit comments

Comments
 (0)
Please sign in to comment.