21
21
import org .apache .kafka .common .TopicPartition ;
22
22
import org .junit .jupiter .api .Test ;
23
23
24
- import java .util .ArrayList ;
25
24
import java .util .Arrays ;
26
25
import java .util .Collection ;
27
26
import java .util .Collections ;
@@ -49,7 +48,7 @@ public void testMirrorCheckpointConnectorDisabled() {
49
48
MirrorCheckpointConfig config = new MirrorCheckpointConfig (
50
49
makeProps ("emit.checkpoints.enabled" , "false" ));
51
50
52
- List <String > knownConsumerGroups = new ArrayList <>();
51
+ Set <String > knownConsumerGroups = new HashSet <>();
53
52
knownConsumerGroups .add (CONSUMER_GROUP );
54
53
// MirrorCheckpointConnector as minimum to run taskConfig()
55
54
MirrorCheckpointConnector connector = new MirrorCheckpointConnector (knownConsumerGroups ,
@@ -65,7 +64,7 @@ public void testMirrorCheckpointConnectorEnabled() {
65
64
MirrorCheckpointConfig config = new MirrorCheckpointConfig (
66
65
makeProps ("emit.checkpoints.enabled" , "true" ));
67
66
68
- List <String > knownConsumerGroups = new ArrayList <>();
67
+ Set <String > knownConsumerGroups = new HashSet <>();
69
68
knownConsumerGroups .add (CONSUMER_GROUP );
70
69
// MirrorCheckpointConnector as minimum to run taskConfig()
71
70
MirrorCheckpointConnector connector = new MirrorCheckpointConnector (knownConsumerGroups ,
@@ -81,7 +80,7 @@ public void testMirrorCheckpointConnectorEnabled() {
81
80
@ Test
82
81
public void testNoConsumerGroup () {
83
82
MirrorCheckpointConfig config = new MirrorCheckpointConfig (makeProps ());
84
- MirrorCheckpointConnector connector = new MirrorCheckpointConnector (new ArrayList <>(), config );
83
+ MirrorCheckpointConnector connector = new MirrorCheckpointConnector (new HashSet <>(), config );
85
84
List <Map <String , String >> output = connector .taskConfigs (1 );
86
85
// expect no task will be created
87
86
assertEquals (0 , output .size (), "ConsumerGroup shouldn't exist" );
@@ -92,7 +91,7 @@ public void testReplicationDisabled() {
92
91
// disable the replication
93
92
MirrorCheckpointConfig config = new MirrorCheckpointConfig (makeProps ("enabled" , "false" ));
94
93
95
- List <String > knownConsumerGroups = new ArrayList <>();
94
+ Set <String > knownConsumerGroups = new HashSet <>();
96
95
knownConsumerGroups .add (CONSUMER_GROUP );
97
96
// MirrorCheckpointConnector as minimum to run taskConfig()
98
97
MirrorCheckpointConnector connector = new MirrorCheckpointConnector (knownConsumerGroups , config );
@@ -106,7 +105,7 @@ public void testReplicationEnabled() {
106
105
// enable the replication
107
106
MirrorCheckpointConfig config = new MirrorCheckpointConfig (makeProps ("enabled" , "true" ));
108
107
109
- List <String > knownConsumerGroups = new ArrayList <>();
108
+ Set <String > knownConsumerGroups = new HashSet <>();
110
109
knownConsumerGroups .add (CONSUMER_GROUP );
111
110
// MirrorCheckpointConnector as minimum to run taskConfig()
112
111
MirrorCheckpointConnector connector = new MirrorCheckpointConnector (knownConsumerGroups , config );
@@ -120,7 +119,7 @@ public void testReplicationEnabled() {
120
119
@ Test
121
120
public void testFindConsumerGroups () throws Exception {
122
121
MirrorCheckpointConfig config = new MirrorCheckpointConfig (makeProps ());
123
- MirrorCheckpointConnector connector = new MirrorCheckpointConnector (Collections .emptyList (), config );
122
+ MirrorCheckpointConnector connector = new MirrorCheckpointConnector (Collections .emptySet (), config );
124
123
connector = spy (connector );
125
124
126
125
Collection <ConsumerGroupListing > groups = Arrays .asList (
@@ -132,21 +131,21 @@ public void testFindConsumerGroups() throws Exception {
132
131
doReturn (true ).when (connector ).shouldReplicateByTopicFilter (anyString ());
133
132
doReturn (true ).when (connector ).shouldReplicateByGroupFilter (anyString ());
134
133
doReturn (offsets ).when (connector ).listConsumerGroupOffsets (anyString ());
135
- List <String > groupFound = connector .findConsumerGroups ();
134
+ Set <String > groupFound = connector .findConsumerGroups ();
136
135
137
136
Set <String > expectedGroups = groups .stream ().map (ConsumerGroupListing ::groupId ).collect (Collectors .toSet ());
138
- assertEquals (expectedGroups , new HashSet <>( groupFound ) ,
137
+ assertEquals (expectedGroups , groupFound ,
139
138
"Expected groups are not the same as findConsumerGroups" );
140
139
141
140
doReturn (false ).when (connector ).shouldReplicateByTopicFilter (anyString ());
142
- List <String > topicFilterGroupFound = connector .findConsumerGroups ();
143
- assertEquals (Collections .emptyList (), topicFilterGroupFound );
141
+ Set <String > topicFilterGroupFound = connector .findConsumerGroups ();
142
+ assertEquals (Collections .emptySet (), topicFilterGroupFound );
144
143
}
145
144
146
145
@ Test
147
146
public void testFindConsumerGroupsInCommonScenarios () throws Exception {
148
147
MirrorCheckpointConfig config = new MirrorCheckpointConfig (makeProps ());
149
- MirrorCheckpointConnector connector = new MirrorCheckpointConnector (Collections .emptyList (), config );
148
+ MirrorCheckpointConnector connector = new MirrorCheckpointConnector (Collections .emptySet (), config );
150
149
connector = spy (connector );
151
150
152
151
Collection <ConsumerGroupListing > groups = Arrays .asList (
@@ -177,8 +176,11 @@ public void testFindConsumerGroupsInCommonScenarios() throws Exception {
177
176
doReturn (offsetsForGroup3 ).when (connector ).listConsumerGroupOffsets ("g3" );
178
177
doReturn (offsetsForGroup4 ).when (connector ).listConsumerGroupOffsets ("g4" );
179
178
180
- List <String > groupFound = connector .findConsumerGroups ();
181
- assertEquals (groupFound , Arrays .asList ("g1" , "g2" ));
179
+ Set <String > groupFound = connector .findConsumerGroups ();
180
+ Set <String > verifiedSet = new HashSet <>();
181
+ verifiedSet .add ("g1" );
182
+ verifiedSet .add ("g2" );
183
+ assertEquals (groupFound , verifiedSet );
182
184
}
183
185
184
186
}
0 commit comments