29
29
import org .apache .kafka .common .TopicIdPartition ;
30
30
import org .apache .kafka .common .TopicPartition ;
31
31
import org .apache .kafka .common .Uuid ;
32
+ import org .apache .kafka .common .errors .InterruptException ;
33
+ import org .apache .kafka .common .errors .WakeupException ;
32
34
import org .apache .kafka .common .message .ConsumerGroupHeartbeatResponseData ;
33
35
import org .apache .kafka .common .message .ConsumerGroupHeartbeatResponseData .Assignment ;
34
36
import org .apache .kafka .common .message .ConsumerGroupHeartbeatResponseData .TopicPartitions ;
96
98
import static org .mockito .Mockito .verify ;
97
99
import static org .mockito .Mockito .when ;
98
100
101
+ @ SuppressWarnings ("ClassDataAbstractionCoupling" )
99
102
public class ConsumerMembershipManagerTest {
100
103
101
104
private static final String GROUP_ID = "test-group" ;
@@ -1738,14 +1741,20 @@ public void testListenerCallbacksBasic() {
1738
1741
1739
1742
@ Test
1740
1743
public void testListenerCallbacksThrowsErrorOnPartitionsRevoked () {
1744
+ testErrorsOnPartitionsRevoked (new WakeupException ());
1745
+ testErrorsOnPartitionsRevoked (new InterruptException ("Intentional onPartitionsRevoked() error" ));
1746
+ testErrorsOnPartitionsRevoked (new IllegalArgumentException ("Intentional onPartitionsRevoked() error" ));
1747
+ }
1748
+
1749
+ private void testErrorsOnPartitionsRevoked (RuntimeException error ) {
1741
1750
// Step 1: set up mocks
1742
1751
String topicName = "topic1" ;
1743
1752
Uuid topicId = Uuid .randomUuid ();
1744
1753
1745
1754
ConsumerMembershipManager membershipManager = createMemberInStableState ();
1746
1755
mockOwnedPartition (membershipManager , topicId , topicName );
1747
1756
CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener (
1748
- Optional .of ( new IllegalArgumentException ( "Intentional onPartitionsRevoked() error" ) ),
1757
+ Optional .ofNullable ( error ),
1749
1758
Optional .empty (),
1750
1759
Optional .empty ()
1751
1760
);
@@ -1792,14 +1801,20 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() {
1792
1801
1793
1802
@ Test
1794
1803
public void testListenerCallbacksThrowsErrorOnPartitionsAssigned () {
1804
+ testErrorsOnPartitionsAssigned (new WakeupException ());
1805
+ testErrorsOnPartitionsAssigned (new InterruptException ("Intentional error" ));
1806
+ testErrorsOnPartitionsAssigned (new IllegalArgumentException ("Intentional error" ));
1807
+ }
1808
+
1809
+ private void testErrorsOnPartitionsAssigned (RuntimeException error ) {
1795
1810
// Step 1: set up mocks
1796
1811
ConsumerMembershipManager membershipManager = createMemberInStableState ();
1797
1812
String topicName = "topic1" ;
1798
1813
Uuid topicId = Uuid .randomUuid ();
1799
1814
mockOwnedPartition (membershipManager , topicId , topicName );
1800
1815
CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener (
1801
1816
Optional .empty (),
1802
- Optional .of ( new IllegalArgumentException ( "Intentional onPartitionsAssigned() error" ) ),
1817
+ Optional .ofNullable ( error ),
1803
1818
Optional .empty ()
1804
1819
);
1805
1820
ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker ();
@@ -1879,7 +1894,7 @@ public void testAddedPartitionsTemporarilyDisabledAwaitingOnPartitionsAssignedCa
1879
1894
true
1880
1895
);
1881
1896
1882
- verify (subscriptionState ).enablePartitionsAwaitingCallback (addedPartitions );
1897
+ verify (subscriptionState ).enablePartitionsAwaitingCallback (assignedPartitions );
1883
1898
}
1884
1899
1885
1900
@ Test
@@ -1915,12 +1930,14 @@ public void testAddedPartitionsNotEnabledAfterFailedOnPartitionsAssignedCallback
1915
1930
1916
1931
@ Test
1917
1932
public void testOnPartitionsLostNoError () {
1918
- testOnPartitionsLost (Optional . empty () );
1933
+ testOnPartitionsLost (null );
1919
1934
}
1920
1935
1921
1936
@ Test
1922
1937
public void testOnPartitionsLostError () {
1923
- testOnPartitionsLost (Optional .of (new KafkaException ("Intentional error for test" )));
1938
+ testOnPartitionsLost (new KafkaException ("Intentional error for test" ));
1939
+ testOnPartitionsLost (new WakeupException ());
1940
+ testOnPartitionsLost (new InterruptException ("Intentional error for test" ));
1924
1941
}
1925
1942
1926
1943
private void assertLeaveGroupDueToExpiredPollAndTransitionToStale (ConsumerMembershipManager membershipManager ) {
@@ -2054,7 +2071,7 @@ private void mockPartitionOwnedAndNewPartitionAdded(String topicName,
2054
2071
receiveAssignment (topicId , Arrays .asList (partitionOwned , partitionAdded ), membershipManager );
2055
2072
}
2056
2073
2057
- private void testOnPartitionsLost (Optional < RuntimeException > lostError ) {
2074
+ private void testOnPartitionsLost (RuntimeException lostError ) {
2058
2075
// Step 1: set up mocks
2059
2076
ConsumerMembershipManager membershipManager = createMemberInStableState ();
2060
2077
String topicName = "topic1" ;
@@ -2063,7 +2080,7 @@ private void testOnPartitionsLost(Optional<RuntimeException> lostError) {
2063
2080
CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener (
2064
2081
Optional .empty (),
2065
2082
Optional .empty (),
2066
- lostError
2083
+ Optional . ofNullable ( lostError )
2067
2084
);
2068
2085
ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker ();
2069
2086
0 commit comments