-
Notifications
You must be signed in to change notification settings - Fork 217
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request #855 from allegro/release-0.12.4
Release 0.12.4
- Loading branch information
Showing
12 changed files
with
214 additions
and
13 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
29 changes: 29 additions & 0 deletions
29
.../allegro/tech/hermes/consumers/consumer/offset/kafka/broker/KafkaConsumerOffsetMover.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,29 @@ | ||
package pl.allegro.tech.hermes.consumers.consumer.offset.kafka.broker; | ||
|
||
import org.apache.kafka.clients.consumer.KafkaConsumer; | ||
import org.apache.kafka.common.TopicPartition; | ||
import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset; | ||
|
||
class KafkaConsumerOffsetMover implements PartitionAssigningAwareRetransmitter.OffsetMover { | ||
private KafkaConsumer consumer; | ||
|
||
KafkaConsumerOffsetMover(KafkaConsumer consumer) { | ||
this.consumer = consumer; | ||
} | ||
|
||
@Override | ||
public void move(SubscriptionPartitionOffset offset) { | ||
try { | ||
consumer.seek(new TopicPartition(offset.getKafkaTopicName().asString(), offset.getPartition()), offset.getOffset()); | ||
} catch (IllegalStateException ex) { | ||
/* | ||
Unfortunately we can't differentiate between different kind of illegal states in KafkaConsumer. | ||
What we are really interested in is IllegalStateException with message containing | ||
"No current assignment for partition" but because this message can change in any minor release | ||
we can just do a leap of fate and interpret IllegalStateException as PartitionNotAssignedException. | ||
Throwing this exception should only cause retransmission to be retried after consumer rebalancing. | ||
*/ | ||
throw new PartitionNotAssignedException(ex); | ||
} | ||
} | ||
} |
85 changes: 85 additions & 0 deletions
85
...h/hermes/consumers/consumer/offset/kafka/broker/PartitionAssigningAwareRetransmitter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
package pl.allegro.tech.hermes.consumers.consumer.offset.kafka.broker; | ||
|
||
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; | ||
import org.apache.kafka.clients.consumer.KafkaConsumer; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
import pl.allegro.tech.hermes.api.SubscriptionName; | ||
import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Collection; | ||
import java.util.List; | ||
import java.util.concurrent.ArrayBlockingQueue; | ||
import java.util.concurrent.BlockingQueue; | ||
|
||
public class PartitionAssigningAwareRetransmitter implements ConsumerRebalanceListener { | ||
private static final Logger logger = LoggerFactory.getLogger(PartitionAssigningAwareRetransmitter.class); | ||
|
||
private final OffsetMover offsetMover; | ||
private final SubscriptionName subscriptionName; | ||
private final BlockingQueue<SubscriptionPartitionOffset> retransmissionQueue; | ||
|
||
public PartitionAssigningAwareRetransmitter(SubscriptionName subscriptionName, int queueSize, KafkaConsumer consumer) { | ||
this(subscriptionName, queueSize, new KafkaConsumerOffsetMover(consumer)); | ||
} | ||
|
||
public PartitionAssigningAwareRetransmitter(SubscriptionName subscriptionName, int queueSize, OffsetMover offsetMover) { | ||
this.offsetMover = offsetMover; | ||
this.subscriptionName = subscriptionName; | ||
this.retransmissionQueue = new ArrayBlockingQueue<>(queueSize); | ||
} | ||
|
||
@Override | ||
public void onPartitionsRevoked(Collection<TopicPartition> partitions) { | ||
// not interesting for retransmission | ||
} | ||
|
||
@Override | ||
public void onPartitionsAssigned(Collection<TopicPartition> partitions) { | ||
if (!retransmissionQueue.isEmpty()) { | ||
logger.info("Detected scheduled retransmission for subscription {}", subscriptionName); | ||
moveScheduledOffsets(); | ||
} | ||
} | ||
|
||
public void moveOffsetOrSchedule(SubscriptionPartitionOffset offset) { | ||
try { | ||
logger.info("Moving offset for subscription {} {}", offset.getSubscriptionName(), offset.toString()); | ||
offsetMover.move(offset); | ||
} catch (PartitionNotAssignedException ex) { | ||
logger.info("Failed to move offset right now, reason: {}", ex.getMessage()); | ||
boolean scheduled = retransmissionQueue.offer(offset); | ||
if (scheduled) { | ||
logger.info("Scheduled retransmission for subscription {} on next rebalance," + | ||
" offset {}", subscriptionName, offset.toString()); | ||
} else { | ||
logger.info("Failed to schedule new retransmission for subscription {}," + | ||
"there is already retransmission scheduled on next rebalance.", subscriptionName); | ||
} | ||
} | ||
} | ||
|
||
public boolean isQueueEmpty() { | ||
return retransmissionQueue.isEmpty(); | ||
} | ||
|
||
private void moveScheduledOffsets() { | ||
List<SubscriptionPartitionOffset> offsets = new ArrayList<>(); | ||
retransmissionQueue.drainTo(offsets); | ||
offsets.forEach(offset -> { | ||
try { | ||
offsetMover.move(offset); | ||
} catch (Exception ex) { | ||
logger.info("Still cannot move offset after rebalance for partition {} for subscription {}," + | ||
" possibly owned by different node", | ||
offset.getPartition(), offset.getSubscriptionName(), ex); | ||
} | ||
}); | ||
} | ||
|
||
interface OffsetMover { | ||
void move(SubscriptionPartitionOffset offset) throws PartitionNotAssignedException; | ||
} | ||
} |
13 changes: 13 additions & 0 deletions
13
...gro/tech/hermes/consumers/consumer/offset/kafka/broker/PartitionNotAssignedException.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,13 @@ | ||
package pl.allegro.tech.hermes.consumers.consumer.offset.kafka.broker; | ||
|
||
import pl.allegro.tech.hermes.common.exception.RetransmissionException; | ||
|
||
public class PartitionNotAssignedException extends RetransmissionException { | ||
public PartitionNotAssignedException() { | ||
super(""); | ||
} | ||
|
||
public PartitionNotAssignedException(Throwable cause) { | ||
super(cause); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
53 changes: 53 additions & 0 deletions
53
...es/consumers/consumer/offset/kafka/broker/PartitionAssigningAwareRetransmitterTest.groovy
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,53 @@ | ||
package pl.allegro.tech.hermes.consumers.consumer.offset.kafka.broker | ||
|
||
import spock.lang.Specification | ||
|
||
import static pl.allegro.tech.hermes.api.SubscriptionName.fromString | ||
import static pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset.subscriptionPartitionOffset | ||
|
||
|
||
class PartitionAssigningAwareRetransmitterTest extends Specification { | ||
|
||
def topic = "group.topic" | ||
def subscriptionName = "$topic\$subscription" | ||
def offsetMover = Mock(PartitionAssigningAwareRetransmitter.OffsetMover) | ||
def offset = subscriptionPartitionOffset(topic, subscriptionName, 1, 1) | ||
|
||
def retransmitter = new PartitionAssigningAwareRetransmitter(fromString(subscriptionName), 10, offsetMover) | ||
|
||
def "should move offset when partitions are already assigned"() { | ||
when: | ||
retransmitter.moveOffsetOrSchedule(offset) | ||
|
||
then: | ||
1 * offsetMover.move(offset) | ||
retransmitter.isQueueEmpty() | ||
} | ||
|
||
def "should schedule retransmission if consumer does not have partitions assigned yet"() { | ||
when: | ||
retransmitter.moveOffsetOrSchedule(offset) | ||
|
||
then: | ||
1 * offsetMover.move(offset) >> { throw new PartitionNotAssignedException() } | ||
!retransmitter.isQueueEmpty() | ||
} | ||
|
||
def "should trigger scheduled retransmission when partitions are assigned"() { | ||
when: | ||
retransmitter.moveOffsetOrSchedule(offset) | ||
retransmitter.onPartitionsAssigned([]) | ||
|
||
then: | ||
2 * offsetMover.move(offset) >> { throw new PartitionNotAssignedException() } | ||
retransmitter.isQueueEmpty() | ||
} | ||
|
||
def "should not move offset after rebalance if nothing was scheduled"() { | ||
when: | ||
retransmitter.onPartitionsAssigned([]) | ||
|
||
then: | ||
0 * offsetMover.move(_) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters