chia7712 commented on code in PR #18089:
URL: https://github.com/apache/kafka/pull/18089#discussion_r1883117990
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java:
##########
@@ -1175,18 +1175,20 @@ private CompletableFuture<Void> assignPartitions(
// Invoke user call back.
CompletableFuture<Void> result =
signalPartitionsAssigned(addedPartitions);
+ // Enable newly added partitions to start fetching and updating
positions for them.
result.whenComplete((__, exception) -> {
if (exception == null) {
- // Enable newly added partitions to start fetching and
updating positions for them.
-
subscriptions.enablePartitionsAwaitingCallback(addedPartitions);
+ // Enable assigned partitions to start fetching and updating
positions for them.
+ subscriptions.enablePartitionsAwaitingCallback(
+
assignedPartitions.stream().map(TopicIdPartition::topicPartition).collect(Collectors.toSet()));
} else {
// Keeping newly added partitions as non-fetchable after the
callback failure.
// They will be retried on the next reconciliation loop, until
it succeeds or the
// broker removes them from the assignment.
if (!addedPartitions.isEmpty()) {
log.warn("Leaving newly assigned partitions {} marked as
non-fetchable and not " +
- "requiring initializing positions after
onPartitionsAssigned callback failed.",
- addedPartitions, exception);
+ "requiring initializing positions after
onPartitionsAssigned callback failed.",
Review Comment:
please revert this change
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java:
##########
@@ -1175,18 +1175,20 @@ private CompletableFuture<Void> assignPartitions(
// Invoke user call back.
CompletableFuture<Void> result =
signalPartitionsAssigned(addedPartitions);
+ // Enable newly added partitions to start fetching and updating
positions for them.
result.whenComplete((__, exception) -> {
if (exception == null) {
- // Enable newly added partitions to start fetching and
updating positions for them.
-
subscriptions.enablePartitionsAwaitingCallback(addedPartitions);
+ // Enable assigned partitions to start fetching and updating
positions for them.
+ subscriptions.enablePartitionsAwaitingCallback(
+
assignedPartitions.stream().map(TopicIdPartition::topicPartition).collect(Collectors.toSet()));
Review Comment:
please add comments for this change
##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java:
##########
@@ -1920,6 +1935,8 @@ public void testOnPartitionsLostNoError() {
@Test
public void testOnPartitionsLostError() {
testOnPartitionsLost(Optional.of(new KafkaException("Intentional error
for test")));
+ testOnPartitionsLost(Optional.of(new WakeupException()));
+ testOnPartitionsLost(Optional.of(new InterruptException("Intentional
error for test")));
Review Comment:
ditto
##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java:
##########
@@ -1737,14 +1740,20 @@ public void testListenerCallbacksBasic() {
@Test
public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() {
+ testErrorsOnPartitionsRevoked(Optional.of(new WakeupException()));
+ testErrorsOnPartitionsRevoked(Optional.of(new
InterruptException("Intentional onPartitionsRevoked() error")));
+ testErrorsOnPartitionsRevoked(Optional.of(new
IllegalArgumentException("Intentional onPartitionsRevoked() error")));
+ }
+
+ private void testErrorsOnPartitionsRevoked(Optional<RuntimeException>
error) {
Review Comment:
we don't need `Optional`, right?
##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java:
##########
@@ -1791,14 +1800,20 @@ public void
testListenerCallbacksThrowsErrorOnPartitionsRevoked() {
@Test
public void testListenerCallbacksThrowsErrorOnPartitionsAssigned() {
+ testErrorsOnPartitionsAssigned(Optional.of(new WakeupException()));
+ testErrorsOnPartitionsAssigned(Optional.of(new
InterruptException("Intentional error")));
+ testErrorsOnPartitionsAssigned(Optional.of(new
IllegalArgumentException("Intentional error")));
+ }
+
+ private void testErrorsOnPartitionsAssigned(Optional<RuntimeException>
error) {
Review Comment:
ditto
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]