forked from apache/kafka
/
DistributedHerderTest.java
3944 lines (3177 loc) · 185 KB
/
DistributedHerderTest.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.runtime.distributed;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.config.ConfigValue;
import org.apache.kafka.common.errors.AuthorizationException;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.connect.errors.AlreadyExistsException;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.errors.NotFoundException;
import org.apache.kafka.connect.runtime.AbstractStatus;
import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
import org.apache.kafka.connect.runtime.ConnectorConfig;
import org.apache.kafka.connect.runtime.ConnectorStatus;
import org.apache.kafka.connect.runtime.Herder;
import org.apache.kafka.connect.runtime.MockConnectMetrics;
import org.apache.kafka.connect.runtime.RestartPlan;
import org.apache.kafka.connect.runtime.RestartRequest;
import org.apache.kafka.connect.runtime.SessionKey;
import org.apache.kafka.connect.runtime.SinkConnectorConfig;
import org.apache.kafka.connect.runtime.SourceConnectorConfig;
import org.apache.kafka.connect.runtime.TargetState;
import org.apache.kafka.connect.runtime.TaskConfig;
import org.apache.kafka.connect.runtime.TaskStatus;
import org.apache.kafka.connect.runtime.TopicStatus;
import org.apache.kafka.connect.runtime.Worker;
import org.apache.kafka.connect.runtime.WorkerConfig;
import org.apache.kafka.connect.runtime.WorkerConfigTransformer;
import org.apache.kafka.connect.runtime.distributed.DistributedHerder.HerderMetrics;
import org.apache.kafka.connect.runtime.isolation.Plugins;
import org.apache.kafka.connect.runtime.rest.InternalRequestSignature;
import org.apache.kafka.connect.runtime.rest.RestClient;
import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos;
import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffset;
import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
import org.apache.kafka.connect.runtime.rest.entities.ConnectorType;
import org.apache.kafka.connect.runtime.rest.entities.Message;
import org.apache.kafka.connect.runtime.rest.entities.TaskInfo;
import org.apache.kafka.connect.runtime.rest.errors.BadRequestException;
import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
import org.apache.kafka.connect.sink.SinkConnector;
import org.apache.kafka.connect.source.ConnectorTransactionBoundaries;
import org.apache.kafka.connect.source.ExactlyOnceSupport;
import org.apache.kafka.connect.source.SourceConnector;
import org.apache.kafka.connect.source.SourceTask;
import org.apache.kafka.connect.storage.ClusterConfigState;
import org.apache.kafka.connect.storage.ConfigBackingStore;
import org.apache.kafka.connect.storage.StatusBackingStore;
import org.apache.kafka.connect.util.Callback;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.apache.kafka.connect.util.FutureCallback;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.junit.MockitoJUnitRunner;
import org.mockito.stubbing.Answer;
import org.mockito.stubbing.OngoingStubbing;
import javax.crypto.SecretKey;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.AbstractExecutorService;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static java.util.Collections.singletonList;
import static javax.ws.rs.core.Response.Status.FORBIDDEN;
import static javax.ws.rs.core.Response.Status.SERVICE_UNAVAILABLE;
import static org.apache.kafka.connect.runtime.AbstractStatus.State.FAILED;
import static org.apache.kafka.connect.runtime.SourceConnectorConfig.ExactlyOnceSupportLevel.REQUIRED;
import static org.apache.kafka.connect.runtime.distributed.ConnectProtocol.CONNECT_PROTOCOL_V0;
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG;
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.INTER_WORKER_KEY_GENERATION_ALGORITHM_DEFAULT;
import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V1;
import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V2;
import static org.apache.kafka.connect.source.SourceTask.TransactionBoundary.CONNECTOR;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.AdditionalMatchers.leq;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyMap;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.ArgumentMatchers.isNull;
import static org.mockito.Mockito.CALLS_REAL_METHODS;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
import static org.mockito.Mockito.withSettings;
@RunWith(MockitoJUnitRunner.StrictStubs.class)
@SuppressWarnings("unchecked")
public class DistributedHerderTest {
private static final Map<String, String> HERDER_CONFIG = new HashMap<>();
static {
HERDER_CONFIG.put(DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "status-topic");
HERDER_CONFIG.put(DistributedConfig.CONFIG_TOPIC_CONFIG, "config-topic");
HERDER_CONFIG.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
HERDER_CONFIG.put(DistributedConfig.GROUP_ID_CONFIG, "connect-test-group");
// The WorkerConfig base class has some required settings without defaults
HERDER_CONFIG.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
HERDER_CONFIG.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
HERDER_CONFIG.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "connect-offsets");
}
private static final String MEMBER_URL = "memberUrl";
private static final String CONN1 = "sourceA";
private static final String CONN2 = "sourceB";
private static final ConnectorTaskId TASK0 = new ConnectorTaskId(CONN1, 0);
private static final ConnectorTaskId TASK1 = new ConnectorTaskId(CONN1, 1);
private static final ConnectorTaskId TASK2 = new ConnectorTaskId(CONN1, 2);
private static final Integer MAX_TASKS = 3;
private static final Map<String, String> CONN1_CONFIG = new HashMap<>();
private static final String FOO_TOPIC = "foo";
private static final String BAR_TOPIC = "bar";
private static final String BAZ_TOPIC = "baz";
static {
CONN1_CONFIG.put(ConnectorConfig.NAME_CONFIG, CONN1);
CONN1_CONFIG.put(ConnectorConfig.TASKS_MAX_CONFIG, MAX_TASKS.toString());
CONN1_CONFIG.put(SinkConnectorConfig.TOPICS_CONFIG, String.join(",", FOO_TOPIC, BAR_TOPIC));
CONN1_CONFIG.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, BogusSourceConnector.class.getName());
}
private static final Map<String, String> CONN1_CONFIG_UPDATED = new HashMap<>(CONN1_CONFIG);
static {
CONN1_CONFIG_UPDATED.put(SinkConnectorConfig.TOPICS_CONFIG, String.join(",", FOO_TOPIC, BAR_TOPIC, BAZ_TOPIC));
}
private static final ConfigInfos CONN1_CONFIG_INFOS =
new ConfigInfos(CONN1, 0, Collections.emptyList(), Collections.emptyList());
private static final Map<String, String> CONN2_CONFIG = new HashMap<>();
static {
CONN2_CONFIG.put(ConnectorConfig.NAME_CONFIG, CONN2);
CONN2_CONFIG.put(ConnectorConfig.TASKS_MAX_CONFIG, MAX_TASKS.toString());
CONN2_CONFIG.put(SinkConnectorConfig.TOPICS_CONFIG, String.join(",", FOO_TOPIC, BAR_TOPIC));
CONN2_CONFIG.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, BogusSourceConnector.class.getName());
}
private static final ConfigInfos CONN2_CONFIG_INFOS =
new ConfigInfos(CONN2, 0, Collections.emptyList(), Collections.emptyList());
private static final ConfigInfos CONN2_INVALID_CONFIG_INFOS =
new ConfigInfos(CONN2, 1, Collections.emptyList(), Collections.emptyList());
private static final Map<String, String> TASK_CONFIG = new HashMap<>();
static {
TASK_CONFIG.put(TaskConfig.TASK_CLASS_CONFIG, BogusSourceTask.class.getName());
}
private static final List<Map<String, String>> TASK_CONFIGS = new ArrayList<>();
static {
TASK_CONFIGS.add(TASK_CONFIG);
TASK_CONFIGS.add(TASK_CONFIG);
TASK_CONFIGS.add(TASK_CONFIG);
}
private static final HashMap<ConnectorTaskId, Map<String, String>> TASK_CONFIGS_MAP = new HashMap<>();
static {
TASK_CONFIGS_MAP.put(TASK0, TASK_CONFIG);
TASK_CONFIGS_MAP.put(TASK1, TASK_CONFIG);
TASK_CONFIGS_MAP.put(TASK2, TASK_CONFIG);
}
private static final ClusterConfigState SNAPSHOT = new ClusterConfigState(
1,
null,
Collections.singletonMap(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED),
TASK_CONFIGS_MAP,
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptySet(),
Collections.emptySet());
private static final ClusterConfigState SNAPSHOT_PAUSED_CONN1 = new ClusterConfigState(
1,
null,
Collections.singletonMap(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.PAUSED),
TASK_CONFIGS_MAP,
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptySet(),
Collections.emptySet());
private static final ClusterConfigState SNAPSHOT_STOPPED_CONN1 = new ClusterConfigState(
1,
null,
Collections.singletonMap(CONN1, 0),
Collections.singletonMap(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STOPPED),
Collections.emptyMap(), // Stopped connectors should have an empty set of task configs
Collections.singletonMap(CONN1, 3),
Collections.singletonMap(CONN1, 10),
Collections.singleton(CONN1),
Collections.emptySet());
private static final ClusterConfigState SNAPSHOT_STOPPED_CONN1_FENCED = new ClusterConfigState(
1,
null,
Collections.singletonMap(CONN1, 0),
Collections.singletonMap(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STOPPED),
Collections.emptyMap(),
Collections.singletonMap(CONN1, 0),
Collections.singletonMap(CONN1, 11),
Collections.emptySet(),
Collections.emptySet());
private static final ClusterConfigState SNAPSHOT_UPDATED_CONN1_CONFIG = new ClusterConfigState(
1,
null,
Collections.singletonMap(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG_UPDATED),
Collections.singletonMap(CONN1, TargetState.STARTED),
TASK_CONFIGS_MAP,
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptySet(),
Collections.emptySet());
private static final String WORKER_ID = "localhost:8083";
private static final String KAFKA_CLUSTER_ID = "I4ZmrWqfT2e-upky_4fdPA";
private static final Runnable EMPTY_RUNNABLE = () -> {
};
@Mock private ConfigBackingStore configBackingStore;
@Mock private StatusBackingStore statusBackingStore;
@Mock private WorkerGroupMember member;
private MockTime time;
private DistributedHerder herder;
private MockConnectMetrics metrics;
@Mock private Worker worker;
@Mock private Callback<Herder.Created<ConnectorInfo>> putConnectorCallback;
@Mock private Plugins plugins;
@Mock private RestClient restClient;
private final CountDownLatch shutdownCalled = new CountDownLatch(1);
private ConfigBackingStore.UpdateListener configUpdateListener;
private WorkerRebalanceListener rebalanceListener;
private ExecutorService herderExecutor;
private Future<?> herderFuture;
private SinkConnectorConfig conn1SinkConfig;
private SinkConnectorConfig conn1SinkConfigUpdated;
private short connectProtocolVersion;
private final SampleConnectorClientConfigOverridePolicy
noneConnectorClientConfigOverridePolicy = new SampleConnectorClientConfigOverridePolicy();
@Before
public void setUp() throws Exception {
time = new MockTime();
metrics = new MockConnectMetrics(time);
when(worker.isSinkConnector(CONN1)).thenReturn(Boolean.TRUE);
AutoCloseable uponShutdown = () -> shutdownCalled.countDown();
// Default to the old protocol unless specified otherwise
connectProtocolVersion = CONNECT_PROTOCOL_V0;
herder = mock(DistributedHerder.class, withSettings().defaultAnswer(CALLS_REAL_METHODS).useConstructor(new DistributedConfig(HERDER_CONFIG),
worker, WORKER_ID, KAFKA_CLUSTER_ID, statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time,
noneConnectorClientConfigOverridePolicy, Collections.emptyList(), null, new AutoCloseable[]{uponShutdown}));
configUpdateListener = herder.new ConfigUpdateListener();
rebalanceListener = herder.new RebalanceListener(time);
conn1SinkConfig = new SinkConnectorConfig(plugins, CONN1_CONFIG);
conn1SinkConfigUpdated = new SinkConnectorConfig(plugins, CONN1_CONFIG_UPDATED);
when(herder.connectorType(anyMap())).thenReturn(ConnectorType.SOURCE);
}
@After
public void tearDown() {
if (metrics != null) metrics.stop();
if (herderExecutor != null) {
herderExecutor.shutdownNow();
herderExecutor = null;
}
}
@Test
public void testJoinAssignment() throws Exception {
// Join group and get assignment
when(member.memberId()).thenReturn("member");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1));
expectConfigRefreshAndSnapshot(SNAPSHOT);
ArgumentCaptor<Callback<TargetState>> onStart = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
onStart.getValue().onCompletion(null, TargetState.STARTED);
return true;
}).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture());
expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS);
when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true);
doNothing().when(member).poll(anyLong());
herder.tick();
time.sleep(1000L);
assertStatistics(3, 1, 100, 1000L);
verifyNoMoreInteractions(member, configBackingStore, statusBackingStore, worker);
}
@Test
public void testRebalance() throws Exception {
// Join group and get assignment
when(member.memberId()).thenReturn("member");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1));
expectConfigRefreshAndSnapshot(SNAPSHOT);
ArgumentCaptor<Callback<TargetState>> onStart = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
onStart.getValue().onCompletion(null, TargetState.STARTED);
return true;
}).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture());
expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS);
when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true);
doNothing().when(member).poll(anyLong());
time.sleep(1000L);
assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY);
herder.tick();
time.sleep(2000L);
assertStatistics(3, 1, 100, 2000);
verify(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any());
verify(worker).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig));
verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED));
// Rebalance and get a new assignment
expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR,
1, Arrays.asList(CONN1), Arrays.asList());
herder.tick();
time.sleep(3000L);
assertStatistics(3, 2, 100, 3000);
// Verify that the connector is started twice but the task is only started once (the first mocked rebalance assigns CONN1 and TASK1,
// the second mocked rebalance revokes CONN1 and TASK1 and (re)assigns CONN1)
verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any());
verify(worker, times(2)).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig));
verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED));
verifyNoMoreInteractions(member, configBackingStore, statusBackingStore, worker);
}
@Test
public void testIncrementalCooperativeRebalanceForNewMember() throws Exception {
connectProtocolVersion = CONNECT_PROTOCOL_V1;
// Join group. First rebalance contains revocations from other members. For the new
// member the assignment should be empty
when(member.memberId()).thenReturn("member");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1);
expectRebalance(1, Collections.emptyList(), Collections.emptyList());
expectConfigRefreshAndSnapshot(SNAPSHOT);
doNothing().when(member).poll(anyLong());
time.sleep(1000L);
assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY);
herder.tick();
// The new member got its assignment
expectRebalance(Collections.emptyList(), Collections.emptyList(),
ConnectProtocol.Assignment.NO_ERROR,
1, Arrays.asList(CONN1), Arrays.asList(TASK1), 0);
// and the new assignment started
ArgumentCaptor<Callback<TargetState>> onStart = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
onStart.getValue().onCompletion(null, TargetState.STARTED);
return true;
}).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture());
expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS);
when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true);
time.sleep(2000L);
assertStatistics(3, 1, 100, 2000);
herder.tick();
time.sleep(3000L);
assertStatistics(3, 2, 100, 3000);
verify(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any());
verify(worker).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig));
verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED));
verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker);
}
@Test
public void testIncrementalCooperativeRebalanceForExistingMember() {
connectProtocolVersion = CONNECT_PROTOCOL_V1;
// Join group. First rebalance contains revocations because a new member joined.
when(member.memberId()).thenReturn("member");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1);
expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1),
ConnectProtocol.Assignment.NO_ERROR, 1,
Collections.emptyList(), Collections.emptyList(), 0);
doNothing().when(member).requestRejoin();
doNothing().when(member).poll(anyLong());
herder.configState = SNAPSHOT;
time.sleep(1000L);
assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY);
herder.tick();
// In the second rebalance the new member gets its assignment and this member has no
// assignments or revocations
expectRebalance(1, Collections.emptyList(), Collections.emptyList());
time.sleep(2000L);
assertStatistics(3, 1, 100, 2000);
herder.tick();
time.sleep(3000L);
assertStatistics(3, 2, 100, 3000);
verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker);
}
@Test
public void testIncrementalCooperativeRebalanceWithDelay() throws Exception {
connectProtocolVersion = CONNECT_PROTOCOL_V1;
// Join group. First rebalance contains some assignments but also a delay, because a
// member was detected missing
int rebalanceDelay = 10_000;
when(member.memberId()).thenReturn("member");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1);
expectRebalance(Collections.emptyList(), Collections.emptyList(),
ConnectProtocol.Assignment.NO_ERROR, 1,
Collections.emptyList(), Arrays.asList(TASK2),
rebalanceDelay);
expectConfigRefreshAndSnapshot(SNAPSHOT);
when(worker.startSourceTask(eq(TASK2), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true);
doAnswer(invocation -> {
time.sleep(9900L);
return null;
}).when(member).poll(anyLong());
// Request to re-join expected because the scheduled rebalance delay has been reached
doNothing().when(member).requestRejoin();
time.sleep(1000L);
assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY);
herder.tick();
// The member got its assignment and revocation
expectRebalance(Collections.emptyList(), Collections.emptyList(),
ConnectProtocol.Assignment.NO_ERROR,
1, Arrays.asList(CONN1), Arrays.asList(TASK1), 0);
// and the new assignment started
ArgumentCaptor<Callback<TargetState>> onStart = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
onStart.getValue().onCompletion(null, TargetState.STARTED);
return true;
}).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture());
expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS);
when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true);
doNothing().when(member).poll(anyLong());
herder.tick();
time.sleep(2000L);
assertStatistics(3, 2, 100, 2000);
verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker);
}
@Test
public void testRebalanceFailedConnector() throws Exception {
// Join group and get assignment
when(member.memberId()).thenReturn("member");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1));
expectConfigRefreshAndSnapshot(SNAPSHOT);
ArgumentCaptor<Callback<TargetState>> onStart = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
onStart.getValue().onCompletion(null, TargetState.STARTED);
return true;
}).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture());
expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS);
when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true);
doNothing().when(member).poll(anyLong());
herder.tick();
time.sleep(1000L);
assertStatistics(3, 1, 100, 1000L);
verify(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any());
verify(worker).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig));
verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED));
// Rebalance and get a new assignment
expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR,
1, Arrays.asList(CONN1), Arrays.asList());
// worker is not running, so we should see no call to connectorTaskConfigs()
expectExecuteTaskReconfiguration(false, null, null);
herder.tick();
time.sleep(2000L);
assertStatistics(3, 2, 100, 2000L);
verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any());
verify(worker).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig));
verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED));
verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker);
}
@Test
public void testRevoke() throws TimeoutException {
revokeAndReassign(false);
}
@Test
public void testIncompleteRebalanceBeforeRevoke() throws TimeoutException {
revokeAndReassign(true);
}
public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutException {
connectProtocolVersion = CONNECT_PROTOCOL_V1;
int configOffset = 1;
// Join group and get initial assignment
when(member.memberId()).thenReturn("member");
when(member.currentProtocolVersion()).thenReturn(connectProtocolVersion);
// The lists need to be mutable because assignments might be removed
expectRebalance(configOffset, new ArrayList<>(singletonList(CONN1)), new ArrayList<>(singletonList(TASK1)));
expectConfigRefreshAndSnapshot(SNAPSHOT);
ArgumentCaptor<Callback<TargetState>> onStart = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
onStart.getValue().onCompletion(null, TargetState.STARTED);
return true;
}).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture());
when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true);
expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS);
doNothing().when(member).poll(anyLong());
herder.tick();
// worker is stable with an existing set of tasks
if (incompleteRebalance) {
// Perform a partial re-balance just prior to the revocation
// bump the configOffset to trigger reading the config topic to the end
configOffset++;
expectRebalance(configOffset, Arrays.asList(), Arrays.asList());
// give it the wrong snapshot, as if we're out of sync/can't reach the broker
expectConfigRefreshAndSnapshot(SNAPSHOT);
doNothing().when(member).requestRejoin();
// tick exits early because we failed, and doesn't do the poll at the end of the method
// the worker did not startWork or reset the rebalanceResolved flag
herder.tick();
}
// Revoke the connector in the next rebalance
expectRebalance(Arrays.asList(CONN1), Arrays.asList(),
ConnectProtocol.Assignment.NO_ERROR, configOffset, Arrays.asList(),
Arrays.asList());
if (incompleteRebalance) {
// Same as SNAPSHOT, except with an updated offset
// Allow the task to read to the end of the topic and complete the rebalance
ClusterConfigState secondSnapshot = new ClusterConfigState(
configOffset,
null,
Collections.singletonMap(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED),
TASK_CONFIGS_MAP,
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptySet(),
Collections.emptySet()
);
expectConfigRefreshAndSnapshot(secondSnapshot);
}
doNothing().when(member).requestRejoin();
herder.tick();
// re-assign the connector back to the same worker to ensure state was cleaned up
expectRebalance(configOffset, Arrays.asList(CONN1), Arrays.asList());
herder.tick();
verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any());
verify(worker, times(2)).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig));
verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED));
verify(worker).stopAndAwaitConnector(CONN1);
// The tick loop where the revoke happens returns early (because there's a subsequent rebalance) and doesn't result in a poll at
// the end of the method
verify(member, times(2)).poll(anyLong());
verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker);
}
@Test
public void testHaltCleansUpWorker() {
herder.halt();
verify(worker).stopAndAwaitConnectors();
verify(worker).stopAndAwaitTasks();
verify(member).stop();
verify(configBackingStore).stop();
verify(statusBackingStore).stop();
verify(worker).stop();
verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore);
}
@Test
public void testCreateConnector() throws Exception {
when(member.memberId()).thenReturn("leader");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
expectConfigRefreshAndSnapshot(SNAPSHOT);
when(statusBackingStore.connectors()).thenReturn(Collections.emptySet());
doNothing().when(member).poll(anyLong());
// Initial rebalance where this member becomes the leader
herder.tick();
// mock the actual validation since its asynchronous nature is difficult to test and should
// be covered sufficiently by the unit tests for the AbstractHerder class
ArgumentCaptor<Callback<ConfigInfos>> validateCallback = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
validateCallback.getValue().onCompletion(null, CONN2_CONFIG_INFOS);
return null;
}).when(herder).validateConnectorConfig(eq(CONN2_CONFIG), validateCallback.capture());
// CONN2 is new, should succeed
doNothing().when(configBackingStore).putConnectorConfig(CONN2, CONN2_CONFIG);
// This will occur just before/during the second tick
doNothing().when(member).ensureActive();
// No immediate action besides this -- change will be picked up via the config log
herder.putConnectorConfig(CONN2, CONN2_CONFIG, false, putConnectorCallback);
// This tick runs the initial herder request, which issues an asynchronous request for
// connector validation
herder.tick();
// Once that validation is complete, another request is added to the herder request queue
// for actually performing the config write; this tick is for that request
herder.tick();
time.sleep(1000L);
assertStatistics(3, 1, 100, 1000L);
ConnectorInfo info = new ConnectorInfo(CONN2, CONN2_CONFIG, Collections.emptyList(), ConnectorType.SOURCE);
verify(putConnectorCallback).onCompletion(isNull(), eq(new Herder.Created<>(true, info)));
verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback);
}
@Test
public void testCreateConnectorConfigBackingStoreError() {
when(member.memberId()).thenReturn("leader");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
expectConfigRefreshAndSnapshot(SNAPSHOT);
when(statusBackingStore.connectors()).thenReturn(Collections.emptySet());
doNothing().when(member).poll(anyLong());
// Initial rebalance where this member becomes the leader
herder.tick();
// Mock the actual validation since its asynchronous nature is difficult to test and should
// be covered sufficiently by the unit tests for the AbstractHerder class
ArgumentCaptor<Callback<ConfigInfos>> validateCallback = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
validateCallback.getValue().onCompletion(null, CONN2_CONFIG_INFOS);
return null;
}).when(herder).validateConnectorConfig(eq(CONN2_CONFIG), validateCallback.capture());
doThrow(new ConnectException("Error writing connector configuration to Kafka"))
.when(configBackingStore).putConnectorConfig(CONN2, CONN2_CONFIG);
// This will occur just before/during the second tick
doNothing().when(member).ensureActive();
herder.putConnectorConfig(CONN2, CONN2_CONFIG, false, putConnectorCallback);
// This tick runs the initial herder request, which issues an asynchronous request for
// connector validation
herder.tick();
// Once that validation is complete, another request is added to the herder request queue
// for actually performing the config write; this tick is for that request
herder.tick();
time.sleep(1000L);
assertStatistics(3, 1, 100, 1000L);
// Verify that the exception thrown during the config backing store write is propagated via the callback
verify(putConnectorCallback).onCompletion(any(ConnectException.class), isNull());
verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback);
}
@Test
public void testCreateConnectorFailedValidation() throws Exception {
when(member.memberId()).thenReturn("leader");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
expectConfigRefreshAndSnapshot(SNAPSHOT);
when(statusBackingStore.connectors()).thenReturn(Collections.emptySet());
doNothing().when(member).poll(anyLong());
HashMap<String, String> config = new HashMap<>(CONN2_CONFIG);
config.remove(ConnectorConfig.NAME_CONFIG);
// Mock the actual validation since its asynchronous nature is difficult to test and should
// be covered sufficiently by the unit tests for the AbstractHerder class
ArgumentCaptor<Callback<ConfigInfos>> validateCallback = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
validateCallback.getValue().onCompletion(null, CONN2_INVALID_CONFIG_INFOS);
return null;
}).when(herder).validateConnectorConfig(eq(config), validateCallback.capture());
herder.putConnectorConfig(CONN2, config, false, putConnectorCallback);
herder.tick();
// We don't need another rebalance to occur
doNothing().when(member).ensureActive();
herder.tick();
time.sleep(1000L);
assertStatistics(3, 1, 100, 1000L);
ArgumentCaptor<Throwable> error = ArgumentCaptor.forClass(Throwable.class);
verify(putConnectorCallback).onCompletion(error.capture(), isNull());
assertTrue(error.getValue() instanceof BadRequestException);
verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback);
}
@Test
public void testConnectorNameConflictsWithWorkerGroupId() {
Map<String, String> config = new HashMap<>(CONN2_CONFIG);
config.put(ConnectorConfig.NAME_CONFIG, "test-group");
SinkConnector connectorMock = mock(SinkConnector.class);
// CONN2 creation should fail because the worker group id (connect-test-group) conflicts with
// the consumer group id we would use for this sink
Map<String, ConfigValue> validatedConfigs = herder.validateSinkConnectorConfig(
connectorMock, SinkConnectorConfig.configDef(), config);
ConfigValue nameConfig = validatedConfigs.get(ConnectorConfig.NAME_CONFIG);
assertEquals(
Collections.singletonList("Consumer group for sink connector named test-group conflicts with Connect worker group connect-test-group"),
nameConfig.errorMessages());
}
@Test
public void testCreateConnectorAlreadyExists() throws Exception {
when(member.memberId()).thenReturn("leader");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
expectConfigRefreshAndSnapshot(SNAPSHOT);
when(statusBackingStore.connectors()).thenReturn(Collections.emptySet());
doNothing().when(member).poll(anyLong());
// mock the actual validation since its asynchronous nature is difficult to test and should
// be covered sufficiently by the unit tests for the AbstractHerder class
ArgumentCaptor<Callback<ConfigInfos>> validateCallback = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS);
return null;
}).when(herder).validateConnectorConfig(eq(CONN1_CONFIG), validateCallback.capture());
herder.putConnectorConfig(CONN1, CONN1_CONFIG, false, putConnectorCallback);
herder.tick();
// We don't need another rebalance to occur
doNothing().when(member).ensureActive();
herder.tick();
time.sleep(1000L);
assertStatistics(3, 1, 100, 1000L);
// CONN1 already exists
verify(putConnectorCallback).onCompletion(any(AlreadyExistsException.class), isNull());
verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback);
}
@Test
public void testDestroyConnector() throws Exception {
when(member.memberId()).thenReturn("leader");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
// Start with one connector
expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true);
expectConfigRefreshAndSnapshot(SNAPSHOT);
ArgumentCaptor<Callback<TargetState>> onStart = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
onStart.getValue().onCompletion(null, TargetState.STARTED);
return true;
}).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture());
expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS);
when(statusBackingStore.connectors()).thenReturn(Collections.emptySet());
doNothing().when(member).poll(anyLong());
// And delete the connector
doNothing().when(configBackingStore).removeConnectorConfig(CONN1);
doNothing().when(putConnectorCallback).onCompletion(null, new Herder.Created<>(false, null));
herder.deleteConnectorConfig(CONN1, putConnectorCallback);
herder.tick();
time.sleep(1000L);
assertStatistics("leaderUrl", false, 3, 1, 100, 1000L);
// The change eventually is reflected to the config topic and the deleted connector and
// tasks are revoked
TopicStatus fooStatus = new TopicStatus(FOO_TOPIC, CONN1, 0, time.milliseconds());
TopicStatus barStatus = new TopicStatus(BAR_TOPIC, CONN1, 0, time.milliseconds());
when(statusBackingStore.getAllTopics(eq(CONN1))).thenReturn(new HashSet<>(Arrays.asList(fooStatus, barStatus)));
doNothing().when(statusBackingStore).deleteTopic(eq(CONN1), eq(FOO_TOPIC));
doNothing().when(statusBackingStore).deleteTopic(eq(CONN1), eq(BAR_TOPIC));
expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1),
ConnectProtocol.Assignment.NO_ERROR, 2, "leader", "leaderUrl",
Collections.emptyList(), Collections.emptyList(), 0, true);
expectConfigRefreshAndSnapshot(ClusterConfigState.EMPTY);
doNothing().when(member).requestRejoin();
configUpdateListener.onConnectorConfigRemove(CONN1); // read updated config that removes the connector
herder.configState = ClusterConfigState.EMPTY;
herder.tick();
time.sleep(1000L);
assertStatistics("leaderUrl", true, 3, 1, 100, 2100L);
verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback);
}
@Test
public void testRestartConnector() throws Exception {
// get the initial assignment
when(member.memberId()).thenReturn("leader");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, singletonList(CONN1), Collections.emptyList(), true);
expectConfigRefreshAndSnapshot(SNAPSHOT);
when(statusBackingStore.connectors()).thenReturn(Collections.emptySet());
doNothing().when(member).poll(anyLong());
ArgumentCaptor<Callback<TargetState>> onStart = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {
onStart.getValue().onCompletion(null, TargetState.STARTED);
return true;
}).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture());
expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS);
// Initial rebalance where this member becomes the leader
herder.tick();
doNothing().when(member).ensureActive();
doNothing().when(worker).stopAndAwaitConnector(CONN1);
FutureCallback<Void> callback = new FutureCallback<>();
herder.restartConnector(CONN1, callback);
herder.tick();
callback.get(1000L, TimeUnit.MILLISECONDS);
verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any());
verify(worker, times(2)).connectorTaskConfigs(eq(CONN1), any());
verify(worker).stopAndAwaitConnector(CONN1);
verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore);
}
@Test
public void testRestartUnknownConnector() throws Exception {
// get the initial assignment
when(member.memberId()).thenReturn("leader");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
expectConfigRefreshAndSnapshot(SNAPSHOT);
when(statusBackingStore.connectors()).thenReturn(Collections.emptySet());
doNothing().when(member).poll(anyLong());
herder.tick();
// now handle the connector restart
doNothing().when(member).ensureActive();
FutureCallback<Void> callback = new FutureCallback<>();
herder.restartConnector(CONN2, callback);
herder.tick();
ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
assertEquals(NotFoundException.class, e.getCause().getClass());
verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore);
}
@Test
public void testRestartConnectorRedirectToLeader() throws Exception {
// get the initial assignment
when(member.memberId()).thenReturn("member");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Collections.emptyList(), Collections.emptyList());
expectConfigRefreshAndSnapshot(SNAPSHOT);
doNothing().when(member).poll(anyLong());
herder.tick();
// now handle the connector restart
doNothing().when(member).ensureActive();
FutureCallback<Void> callback = new FutureCallback<>();
herder.restartConnector(CONN1, callback);
herder.tick();
ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
assertEquals(NotLeaderException.class, e.getCause().getClass());
verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore);
}
@Test
public void testRestartConnectorRedirectToOwner() throws Exception {
// get the initial assignment
when(member.memberId()).thenReturn("leader");
when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0);
expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
expectConfigRefreshAndSnapshot(SNAPSHOT);
when(statusBackingStore.connectors()).thenReturn(Collections.emptySet());
doNothing().when(member).poll(anyLong());
herder.tick();
// now handle the connector restart
doNothing().when(member).ensureActive();
String ownerUrl = "ownerUrl";
when(member.ownerUrl(CONN1)).thenReturn(ownerUrl);
time.sleep(1000L);
assertStatistics(3, 1, 100, 1000L);
FutureCallback<Void> callback = new FutureCallback<>();