@@ -1825,7 +1825,7 @@ public void testPollDoesNotAddNewAsyncPollEventWhenOneIsAlreadyInFlight() {
18251825 }
18261826
18271827 /**
1828- * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer, Predicate) processBackgroundEvents}
1828+ * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer, Predicate, boolean ) processBackgroundEvents}
18291829 * handles the case where the {@link Future} takes a bit of time to complete, but does within the timeout.
18301830 */
18311831 @ Test
@@ -1851,14 +1851,14 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception {
18511851 return null ;
18521852 }).when (future ).get (any (Long .class ), any (TimeUnit .class ));
18531853
1854- consumer .processBackgroundEvents (future , timer , e -> false );
1854+ consumer .processBackgroundEvents (future , timer , e -> false , false );
18551855
18561856 // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries.
18571857 assertEquals (800 , timer .remainingMs ());
18581858 }
18591859
18601860 /**
1861- * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer, Predicate) processBackgroundEvents}
1861+ * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer, Predicate, boolean ) processBackgroundEvents}
18621862 * handles the case where the {@link Future} is already complete when invoked, so it doesn't have to wait.
18631863 */
18641864 @ Test
@@ -1869,15 +1869,15 @@ public void testProcessBackgroundEventsWithoutDelay() {
18691869 // Create a future that is already completed.
18701870 CompletableFuture <?> future = CompletableFuture .completedFuture (null );
18711871
1872- consumer .processBackgroundEvents (future , timer , e -> false );
1872+ consumer .processBackgroundEvents (future , timer , e -> false , false );
18731873
18741874 // Because we didn't need to perform a timed get, we should still have every last millisecond
18751875 // of our initial timeout.
18761876 assertEquals (1000 , timer .remainingMs ());
18771877 }
18781878
18791879 /**
1880- * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer, Predicate) processBackgroundEvents}
1880+ * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer, Predicate, boolean ) processBackgroundEvents}
18811881 * handles the case where the {@link Future} does not complete within the timeout.
18821882 */
18831883 @ Test
@@ -1892,7 +1892,7 @@ public void testProcessBackgroundEventsTimesOut() throws Exception {
18921892 throw new java .util .concurrent .TimeoutException ("Intentional timeout" );
18931893 }).when (future ).get (any (Long .class ), any (TimeUnit .class ));
18941894
1895- assertThrows (TimeoutException .class , () -> consumer .processBackgroundEvents (future , timer , e -> false ));
1895+ assertThrows (TimeoutException .class , () -> consumer .processBackgroundEvents (future , timer , e -> false , false ));
18961896
18971897 // Because we forced our mocked future to continuously time out, we should have no time remaining.
18981898 assertEquals (0 , timer .remainingMs ());
@@ -1993,6 +1993,54 @@ public void testUnsubscribeDoesNotCommitOffsetsEvenWithAutoCommitEnabled() {
19931993 verify (applicationEventHandler , never ()).add (ArgumentMatchers .isA (CommitOnCloseEvent .class ));
19941994 }
19951995
1996+ /**
1997+ * Test to ensure that assignment updates are not applied while unsubscribing
1998+ * (it would cause an IllegalArgumentException when calling unsubscribe()).
1999+ * Validates the fix for KAFKA-20428.
2000+ */
2001+ @ Test
2002+ public void testUnsubscribeWithPendingPartitionsAssignedEvent () {
2003+ consumer = newConsumer (requiredConsumerConfigAndGroupId ("consumerGroup" ));
2004+ completeTopicSubscriptionChangeEventSuccessfully ();
2005+ consumer .subscribe (singletonList ("topic" ));
2006+ completeUnsubscribeApplicationEventSuccessfully ();
2007+
2008+ // Add PartitionsAssignedEvent to the background queue (simulating an ongoing reconciliation
2009+ // that completed just before unsubscribe was called)
2010+ PartitionsAssignedEvent assignedEvent = new PartitionsAssignedEvent (Set .of (), new TreeSet <>(TOPIC_PARTITION_COMPARATOR ));
2011+ backgroundEventQueue .add (assignedEvent );
2012+
2013+ // The call to unsubscribe should complete successfully (PartitionsAssignedEvent not processed and completed exceptionally)
2014+ assertDoesNotThrow (() -> consumer .unsubscribe ());
2015+ verify (applicationEventHandler , never ().description ("Reconciled assignment updates shouldn't be processed while unsubscribing" ))
2016+ .addAndGet (any (ApplyAssignmentEvent .class ));
2017+ assertTrue (assignedEvent .future ().isCompletedExceptionally ());
2018+ }
2019+
2020+ /**
2021+ * Streams equivalent of {@link #testUnsubscribeWithPendingPartitionsAssignedEvent()}.
2022+ */
2023+ @ Test
2024+ public void testUnsubscribeWithPendingStreamsTasksAssignedEvent () {
2025+ consumer = newConsumer (requiredConsumerConfigAndGroupId ("consumerGroup" ));
2026+ completeTopicSubscriptionChangeEventSuccessfully ();
2027+ consumer .subscribe (singletonList ("topic" ));
2028+ completeUnsubscribeApplicationEventSuccessfully ();
2029+
2030+ // Add StreamsTasksAssignedEvent to the background queue (simulating an ongoing reconciliation
2031+ // that completed just before unsubscribe was called)
2032+ StreamsTasksAssignedEvent assignedEvent = new StreamsTasksAssignedEvent (
2033+ new TreeSet <>(TOPIC_PARTITION_COMPARATOR ), new TreeSet <>(TOPIC_PARTITION_COMPARATOR ),
2034+ new StreamsRebalanceData .Assignment (Set .of (), Set .of (), Set .of (), true ));
2035+ backgroundEventQueue .add (assignedEvent );
2036+
2037+ // The call to unsubscribe should complete successfully (StreamsTasksAssignedEvent not processed and completed exceptionally)
2038+ assertDoesNotThrow (() -> consumer .unsubscribe ());
2039+ verify (applicationEventHandler , never ().description ("Reconciled assignment updates shouldn't be processed while unsubscribing" ))
2040+ .addAndGet (any (ApplyAssignmentEvent .class ));
2041+ assertTrue (assignedEvent .future ().isCompletedExceptionally ());
2042+ }
2043+
19962044 @ Test
19972045 public void testSeekToBeginning () {
19982046 Collection <TopicPartition > topics = Collections .singleton (new TopicPartition ("test" , 0 ));
0 commit comments