-
Notifications
You must be signed in to change notification settings - Fork 13.6k
/
DistributedHerder.java
2591 lines (2343 loc) · 129 KB
/
DistributedHerder.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.KafkaFuture;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigValue;
import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.CumulativeSum;
import org.apache.kafka.common.metrics.stats.Max;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.ThreadUtils;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy;
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.AbstractHerder;
import org.apache.kafka.connect.runtime.CloseableConnectorContext;
import org.apache.kafka.connect.runtime.ConnectMetrics;
import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
import org.apache.kafka.connect.runtime.ConnectMetricsRegistry;
import org.apache.kafka.connect.runtime.ConnectorConfig;
import org.apache.kafka.connect.runtime.HerderConnectorContext;
import org.apache.kafka.connect.runtime.HerderRequest;
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.TaskStatus;
import org.apache.kafka.connect.runtime.Worker;
import org.apache.kafka.connect.storage.PrivilegedWriteException;
import org.apache.kafka.connect.runtime.rest.InternalRequestSignature;
import org.apache.kafka.connect.runtime.rest.RestClient;
import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
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.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.ConnectUtils;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.apache.kafka.connect.util.FutureCallback;
import org.apache.kafka.connect.util.SinkUtils;
import org.slf4j.Logger;
import javax.crypto.KeyGenerator;
import javax.crypto.SecretKey;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriBuilder;
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.NavigableSet;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG;
import static org.apache.kafka.connect.runtime.distributed.ConnectProtocol.CONNECT_PROTOCOL_V0;
import static org.apache.kafka.connect.runtime.distributed.ConnectProtocolCompatibility.EAGER;
import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V1;
import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V2;
/**
* <p>
* Distributed "herder" that coordinates with other workers to spread work across multiple processes.
* </p>
* <p>
* Under the hood, this is implemented as a group managed by Kafka's group membership facilities (i.e. the generalized
* group/consumer coordinator). Each instance of DistributedHerder joins the group and indicates what it's current
* configuration state is (where it is in the configuration log). The group coordinator selects one member to take
* this information and assign each instance a subset of the active connectors & tasks to execute. This assignment
* is currently performed in a simple round-robin fashion, but this is not guaranteed -- the herder may also choose
* to, e.g., use a sticky assignment to avoid the usual start/stop costs associated with connectors and tasks. Once
* an assignment is received, the DistributedHerder simply runs its assigned connectors and tasks in a Worker.
* </p>
* <p>
* In addition to distributing work, the DistributedHerder uses the leader determined during the work assignment
* to select a leader for this generation of the group who is responsible for other tasks that can only be performed
* by a single node at a time. Most importantly, this includes writing updated configurations for connectors and tasks,
* (and therefore, also for creating, destroy, and scaling up/down connectors).
* </p>
* <p>
* The DistributedHerder uses a single thread for most of its processing. This includes processing
* config changes, handling task rebalances and serving requests from the HTTP layer. The latter are pushed
* into a queue until the thread has time to handle them. A consequence of this is that requests can get blocked
* behind a worker rebalance. When the herder knows that a rebalance is expected, it typically returns an error
* immediately to the request, but this is not always possible (in particular when another worker has requested
* the rebalance). Similar to handling HTTP requests, config changes which are observed asynchronously by polling
* the config log are batched for handling in the work thread.
* </p>
*/
public class DistributedHerder extends AbstractHerder implements Runnable {
private static final AtomicInteger CONNECT_CLIENT_ID_SEQUENCE = new AtomicInteger(1);
private final Logger log;
private static final long FORWARD_REQUEST_SHUTDOWN_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(10);
private static final long START_AND_STOP_SHUTDOWN_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(1);
private static final long RECONFIGURE_CONNECTOR_TASKS_BACKOFF_MS = 250;
private static final long CONFIG_TOPIC_WRITE_PRIVILEGES_BACKOFF_MS = 250;
private static final int START_STOP_THREAD_POOL_SIZE = 8;
private static final short BACKOFF_RETRIES = 5;
private final AtomicLong requestSeqNum = new AtomicLong();
private final Time time;
private final HerderMetrics herderMetrics;
private final List<AutoCloseable> uponShutdown;
private final String workerGroupId;
private final int workerSyncTimeoutMs;
private final long workerTasksShutdownTimeoutMs;
private final int workerUnsyncBackoffMs;
private final int keyRotationIntervalMs;
private final String requestSignatureAlgorithm;
private final List<String> keySignatureVerificationAlgorithms;
private final KeyGenerator keyGenerator;
// Visible for testing
ExecutorService forwardRequestExecutor;
private final ExecutorService herderExecutor;
private final ExecutorService startAndStopExecutor;
private final WorkerGroupMember member;
private final AtomicBoolean stopping;
private final boolean isTopicTrackingEnabled;
// Track enough information about the current membership state to be able to determine which requests via the API
// and the from other nodes are safe to process
private boolean rebalanceResolved;
private ExtendedAssignment runningAssignment = ExtendedAssignment.empty();
private final Set<ConnectorTaskId> tasksToRestart = new HashSet<>();
// visible for testing
ExtendedAssignment assignment;
private boolean canReadConfigs;
// visible for testing
protected ClusterConfigState configState;
// To handle most external requests, like creating or destroying a connector, we can use a generic request where
// the caller specifies all the code that should be executed.
final NavigableSet<DistributedHerderRequest> requests = new ConcurrentSkipListSet<>();
// Config updates can be collected and applied together when possible. Also, we need to take care to rebalance when
// needed (e.g. task reconfiguration, which requires everyone to coordinate offset commits).
private Set<String> connectorConfigUpdates = new HashSet<>();
private Set<ConnectorTaskId> taskConfigUpdates = new HashSet<>();
// Similarly collect target state changes (when observed by the config storage listener) for handling in the
// herder's main thread.
private Set<String> connectorTargetStateChanges = new HashSet<>();
// Access to this map is protected by the herder's monitor
private final Map<String, ZombieFencing> activeZombieFencings = new HashMap<>();
private boolean needsReconfigRebalance;
private volatile boolean fencedFromConfigTopic;
private volatile int generation;
private volatile long scheduledRebalance;
private volatile SecretKey sessionKey;
private volatile long keyExpiration;
private short currentProtocolVersion;
private short backoffRetries;
// visible for testing
// The latest pending restart request for each named connector
final Map<String, RestartRequest> pendingRestartRequests = new HashMap<>();
// The thread that the herder's tick loop runs on. Would be final, but cannot be set in the constructor,
// and it's also useful to be able to modify it for testing
Thread herderThread;
private final DistributedConfig config;
/**
* Create a herder that will form a Connect cluster with other {@link DistributedHerder} instances (in this or other JVMs)
* that have the same group ID.
*
* @param config the configuration for the worker; may not be null
* @param time the clock to use; may not be null
* @param worker the {@link Worker} instance to use; may not be null
* @param kafkaClusterId the identifier of the Kafka cluster to use for internal topics; may not be null
* @param statusBackingStore the backing store for statuses; may not be null
* @param configBackingStore the backing store for connector configurations; may not be null
* @param restUrl the URL of this herder's REST API; may not be null
* @param connectorClientConfigOverridePolicy the policy specifying the client configuration properties that may be overridden
* in connector configurations; may not be null
* @param uponShutdown any {@link AutoCloseable} objects that should be closed when this herder is {@link #stop() stopped},
* after all services and resources owned by this herder are stopped
*/
public DistributedHerder(DistributedConfig config,
Time time,
Worker worker,
String kafkaClusterId,
StatusBackingStore statusBackingStore,
ConfigBackingStore configBackingStore,
String restUrl,
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
AutoCloseable... uponShutdown) {
this(config, worker, worker.workerId(), kafkaClusterId, statusBackingStore, configBackingStore, null, restUrl, worker.metrics(),
time, connectorClientConfigOverridePolicy, uponShutdown);
configBackingStore.setUpdateListener(new ConfigUpdateListener());
}
// visible for testing
DistributedHerder(DistributedConfig config,
Worker worker,
String workerId,
String kafkaClusterId,
StatusBackingStore statusBackingStore,
ConfigBackingStore configBackingStore,
WorkerGroupMember member,
String restUrl,
ConnectMetrics metrics,
Time time,
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
AutoCloseable... uponShutdown) {
super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy);
this.time = time;
this.herderMetrics = new HerderMetrics(metrics);
this.workerGroupId = config.getString(DistributedConfig.GROUP_ID_CONFIG);
this.workerSyncTimeoutMs = config.getInt(DistributedConfig.WORKER_SYNC_TIMEOUT_MS_CONFIG);
this.workerTasksShutdownTimeoutMs = config.getLong(DistributedConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG);
this.workerUnsyncBackoffMs = config.getInt(DistributedConfig.WORKER_UNSYNC_BACKOFF_MS_CONFIG);
this.requestSignatureAlgorithm = config.getString(DistributedConfig.INTER_WORKER_SIGNATURE_ALGORITHM_CONFIG);
this.keyRotationIntervalMs = config.getInt(DistributedConfig.INTER_WORKER_KEY_TTL_MS_CONFIG);
this.keySignatureVerificationAlgorithms = config.getList(DistributedConfig.INTER_WORKER_VERIFICATION_ALGORITHMS_CONFIG);
this.keyGenerator = config.getInternalRequestKeyGenerator();
this.isTopicTrackingEnabled = config.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG);
this.uponShutdown = Arrays.asList(uponShutdown);
String clientIdConfig = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG);
String clientId = clientIdConfig.length() <= 0 ? "connect-" + CONNECT_CLIENT_ID_SEQUENCE.getAndIncrement() : clientIdConfig;
LogContext logContext = new LogContext("[Worker clientId=" + clientId + ", groupId=" + this.workerGroupId + "] ");
log = logContext.logger(DistributedHerder.class);
this.member = member != null
? member
: new WorkerGroupMember(config, restUrl, this.configBackingStore,
new RebalanceListener(time), time, clientId, logContext);
this.herderExecutor = new ThreadPoolExecutor(1, 1, 0L,
TimeUnit.MILLISECONDS,
new LinkedBlockingDeque<>(1),
ThreadUtils.createThreadFactory(
this.getClass().getSimpleName() + "-" + clientId + "-%d", false));
this.forwardRequestExecutor = Executors.newFixedThreadPool(1,
ThreadUtils.createThreadFactory(
"ForwardRequestExecutor-" + clientId + "-%d", false));
this.startAndStopExecutor = Executors.newFixedThreadPool(START_STOP_THREAD_POOL_SIZE,
ThreadUtils.createThreadFactory(
"StartAndStopExecutor-" + clientId + "-%d", false));
this.config = config;
stopping = new AtomicBoolean(false);
configState = ClusterConfigState.EMPTY;
rebalanceResolved = true; // If we still need to follow up after a rebalance occurred, starting up tasks
needsReconfigRebalance = false;
fencedFromConfigTopic = false;
canReadConfigs = true; // We didn't try yet, but Configs are readable until proven otherwise
scheduledRebalance = Long.MAX_VALUE;
keyExpiration = Long.MAX_VALUE;
sessionKey = null;
backoffRetries = BACKOFF_RETRIES;
currentProtocolVersion = ConnectProtocolCompatibility.compatibility(
config.getString(DistributedConfig.CONNECT_PROTOCOL_CONFIG)
).protocolVersion();
if (!internalRequestValidationEnabled(currentProtocolVersion)) {
log.warn(
"Internal request verification will be disabled for this cluster as this worker's {} configuration has been set to '{}'. "
+ "If this is not intentional, either remove the '{}' configuration from the worker config file or change its value "
+ "to '{}'. If this configuration is left as-is, the cluster will be insecure; for more information, see KIP-507: "
+ "https://cwiki.apache.org/confluence/display/KAFKA/KIP-507%3A+Securing+Internal+Connect+REST+Endpoints",
DistributedConfig.CONNECT_PROTOCOL_CONFIG,
config.getString(DistributedConfig.CONNECT_PROTOCOL_CONFIG),
DistributedConfig.CONNECT_PROTOCOL_CONFIG,
ConnectProtocolCompatibility.SESSIONED.name()
);
}
}
@Override
public void start() {
this.herderExecutor.submit(this);
}
@Override
public void run() {
try {
log.info("Herder starting");
herderThread = Thread.currentThread();
startServices();
log.info("Herder started");
running = true;
while (!stopping.get()) {
tick();
}
halt();
log.info("Herder stopped");
herderMetrics.close();
} catch (Throwable t) {
log.error("Uncaught exception in herder work thread, exiting: ", t);
Exit.exit(1);
} finally {
running = false;
}
}
// public for testing
public void tick() {
// The main loop does two primary things: 1) drive the group membership protocol, responding to rebalance events
// as they occur, and 2) handle external requests targeted at the leader. All the "real" work of the herder is
// performed in this thread, which keeps synchronization straightforward at the cost of some operations possibly
// blocking up this thread (especially those in callbacks due to rebalance events).
try {
// if we failed to read to end of log before, we need to make sure the issue was resolved before joining group
// Joining and immediately leaving for failure to read configs is exceedingly impolite
if (!canReadConfigs) {
if (readConfigToEnd(workerSyncTimeoutMs)) {
canReadConfigs = true;
} else {
return; // Safe to return and tick immediately because readConfigToEnd will do the backoff for us
}
}
log.debug("Ensuring group membership is still active");
member.ensureActive();
// Ensure we're in a good state in our group. If not restart and everything should be setup to rejoin
if (!handleRebalanceCompleted()) return;
} catch (WakeupException e) {
// May be due to a request from another thread, or might be stopping. If the latter, we need to check the
// flag immediately. If the former, we need to re-run the ensureActive call since we can't handle requests
// unless we're in the group.
log.trace("Woken up while ensure group membership is still active");
return;
}
if (fencedFromConfigTopic) {
if (isLeader()) {
// We were accidentally fenced out, possibly by a zombie leader
try {
log.debug("Reclaiming write privileges for config topic after being fenced out");
configBackingStore.claimWritePrivileges();
fencedFromConfigTopic = false;
log.debug("Successfully reclaimed write privileges for config topic after being fenced out");
} catch (Exception e) {
log.warn("Unable to claim write privileges for config topic. Will backoff and possibly retry if still the leader", e);
backoff(CONFIG_TOPIC_WRITE_PRIVILEGES_BACKOFF_MS);
return;
}
} else {
log.trace("Relinquished write privileges for config topic after being fenced out, since worker is no longer the leader of the cluster");
// We were meant to be fenced out because we fell out of the group and a new leader was elected
fencedFromConfigTopic = false;
}
}
long now = time.milliseconds();
if (checkForKeyRotation(now)) {
log.debug("Distributing new session key");
keyExpiration = Long.MAX_VALUE;
try {
SessionKey newSessionKey = new SessionKey(keyGenerator.generateKey(), now);
writeToConfigTopicAsLeader(() -> configBackingStore.putSessionKey(newSessionKey));
} catch (Exception e) {
log.info("Failed to write new session key to config topic; forcing a read to the end of the config topic before possibly retrying", e);
canReadConfigs = false;
return;
}
}
// Process any external requests
// TODO: Some of these can be performed concurrently or even optimized away entirely.
// For example, if three different connectors are slated to be restarted, it's fine to
// restart all three at the same time instead.
// Another example: if multiple configurations are submitted for the same connector,
// the only one that actually has to be written to the config topic is the
// most-recently one.
long nextRequestTimeoutMs = Long.MAX_VALUE;
while (true) {
final DistributedHerderRequest next = peekWithoutException();
if (next == null) {
break;
} else if (now >= next.at) {
requests.pollFirst();
} else {
nextRequestTimeoutMs = next.at - now;
break;
}
runRequest(next.action(), next.callback());
}
// Process all pending connector restart requests
processRestartRequests();
if (scheduledRebalance < Long.MAX_VALUE) {
nextRequestTimeoutMs = Math.min(nextRequestTimeoutMs, Math.max(scheduledRebalance - now, 0));
rebalanceResolved = false;
log.debug("Scheduled rebalance at: {} (now: {} nextRequestTimeoutMs: {}) ",
scheduledRebalance, now, nextRequestTimeoutMs);
}
if (isLeader() && internalRequestValidationEnabled() && keyExpiration < Long.MAX_VALUE) {
nextRequestTimeoutMs = Math.min(nextRequestTimeoutMs, Math.max(keyExpiration - now, 0));
log.debug("Scheduled next key rotation at: {} (now: {} nextRequestTimeoutMs: {}) ",
keyExpiration, now, nextRequestTimeoutMs);
}
// Process any configuration updates
AtomicReference<Set<String>> connectorConfigUpdatesCopy = new AtomicReference<>();
AtomicReference<Set<String>> connectorTargetStateChangesCopy = new AtomicReference<>();
AtomicReference<Set<ConnectorTaskId>> taskConfigUpdatesCopy = new AtomicReference<>();
boolean shouldReturn;
if (member.currentProtocolVersion() == CONNECT_PROTOCOL_V0) {
shouldReturn = updateConfigsWithEager(connectorConfigUpdatesCopy,
connectorTargetStateChangesCopy);
// With eager protocol we should return immediately if needsReconfigRebalance has
// been set to retain the old workflow
if (shouldReturn) {
return;
}
if (connectorConfigUpdatesCopy.get() != null) {
processConnectorConfigUpdates(connectorConfigUpdatesCopy.get());
}
if (connectorTargetStateChangesCopy.get() != null) {
processTargetStateChanges(connectorTargetStateChangesCopy.get());
}
} else {
shouldReturn = updateConfigsWithIncrementalCooperative(connectorConfigUpdatesCopy,
connectorTargetStateChangesCopy, taskConfigUpdatesCopy);
if (connectorConfigUpdatesCopy.get() != null) {
processConnectorConfigUpdates(connectorConfigUpdatesCopy.get());
}
if (connectorTargetStateChangesCopy.get() != null) {
processTargetStateChanges(connectorTargetStateChangesCopy.get());
}
if (taskConfigUpdatesCopy.get() != null) {
processTaskConfigUpdatesWithIncrementalCooperative(taskConfigUpdatesCopy.get());
}
if (shouldReturn) {
return;
}
}
// Let the group take any actions it needs to
try {
log.trace("Polling for group activity; will wait for {}ms or until poll is interrupted by "
+ "either config backing store updates or a new external request",
nextRequestTimeoutMs);
member.poll(nextRequestTimeoutMs);
// Ensure we're in a good state in our group. If not restart and everything should be setup to rejoin
handleRebalanceCompleted();
} catch (WakeupException e) { // FIXME should not be WakeupException
log.trace("Woken up while polling for group activity");
// Ignore. Just indicates we need to check the exit flag, for requested actions, etc.
}
}
private boolean checkForKeyRotation(long now) {
SecretKey key;
long expiration;
synchronized (this) {
// This happens on startup; the snapshot contains the session key,
// but no callback in the config update listener has been fired for it yet.
if (sessionKey == null && configState.sessionKey() != null) {
sessionKey = configState.sessionKey().key();
keyExpiration = configState.sessionKey().creationTimestamp() + keyRotationIntervalMs;
}
key = sessionKey;
expiration = keyExpiration;
}
if (internalRequestValidationEnabled()) {
if (isLeader()) {
if (key == null) {
log.debug("Internal request signing is enabled but no session key has been distributed yet. "
+ "Distributing new key now.");
return true;
} else if (expiration <= now) {
log.debug("Existing key has expired. Distributing new key now.");
return true;
} else if (!key.getAlgorithm().equals(keyGenerator.getAlgorithm())
|| key.getEncoded().length != keyGenerator.generateKey().getEncoded().length) {
log.debug("Previously-distributed key uses different algorithm/key size "
+ "than required by current worker configuration. Distributing new key now.");
return true;
}
}
}
return false;
}
private synchronized boolean updateConfigsWithEager(AtomicReference<Set<String>> connectorConfigUpdatesCopy,
AtomicReference<Set<String>> connectorTargetStateChangesCopy) {
// This branch is here to avoid creating a snapshot if not needed
if (needsReconfigRebalance
|| !connectorConfigUpdates.isEmpty()
|| !connectorTargetStateChanges.isEmpty()) {
log.trace("Handling config updates with eager rebalancing");
// Connector reconfigs only need local updates since there is no coordination between workers required.
// However, if connectors were added or removed, work needs to be rebalanced since we have more work
// items to distribute among workers.
configState = configBackingStore.snapshot();
if (needsReconfigRebalance) {
// Task reconfigs require a rebalance. Request the rebalance, clean out state, and then restart
// this loop, which will then ensure the rebalance occurs without any other requests being
// processed until it completes.
log.debug("Requesting rebalance due to reconfiguration of tasks (needsReconfigRebalance: {})",
needsReconfigRebalance);
member.requestRejoin();
needsReconfigRebalance = false;
// Any connector config updates or target state changes will be addressed during the rebalance too
connectorConfigUpdates.clear();
connectorTargetStateChanges.clear();
return true;
} else {
if (!connectorConfigUpdates.isEmpty()) {
// We can't start/stop while locked since starting connectors can cause task updates that will
// require writing configs, which in turn make callbacks into this class from another thread that
// require acquiring a lock. This leads to deadlock. Instead, just copy the info we need and process
// the updates after unlocking.
connectorConfigUpdatesCopy.set(connectorConfigUpdates);
connectorConfigUpdates = new HashSet<>();
}
if (!connectorTargetStateChanges.isEmpty()) {
// Similarly for target state changes which can cause connectors to be restarted
connectorTargetStateChangesCopy.set(connectorTargetStateChanges);
connectorTargetStateChanges = new HashSet<>();
}
}
} else {
log.trace("Skipping config updates with eager rebalancing "
+ "since no config rebalance is required "
+ "and there are no connector config, task config, or target state changes pending");
}
return false;
}
private synchronized boolean updateConfigsWithIncrementalCooperative(AtomicReference<Set<String>> connectorConfigUpdatesCopy,
AtomicReference<Set<String>> connectorTargetStateChangesCopy,
AtomicReference<Set<ConnectorTaskId>> taskConfigUpdatesCopy) {
boolean retValue = false;
// This branch is here to avoid creating a snapshot if not needed
if (needsReconfigRebalance
|| !connectorConfigUpdates.isEmpty()
|| !connectorTargetStateChanges.isEmpty()
|| !taskConfigUpdates.isEmpty()) {
log.trace("Handling config updates with incremental cooperative rebalancing");
// Connector reconfigs only need local updates since there is no coordination between workers required.
// However, if connectors were added or removed, work needs to be rebalanced since we have more work
// items to distribute among workers.
configState = configBackingStore.snapshot();
if (needsReconfigRebalance) {
log.debug("Requesting rebalance due to reconfiguration of tasks (needsReconfigRebalance: {})",
needsReconfigRebalance);
member.requestRejoin();
needsReconfigRebalance = false;
retValue = true;
}
if (!connectorConfigUpdates.isEmpty()) {
// We can't start/stop while locked since starting connectors can cause task updates that will
// require writing configs, which in turn make callbacks into this class from another thread that
// require acquiring a lock. This leads to deadlock. Instead, just copy the info we need and process
// the updates after unlocking.
connectorConfigUpdatesCopy.set(connectorConfigUpdates);
connectorConfigUpdates = new HashSet<>();
}
if (!connectorTargetStateChanges.isEmpty()) {
// Similarly for target state changes which can cause connectors to be restarted
connectorTargetStateChangesCopy.set(connectorTargetStateChanges);
connectorTargetStateChanges = new HashSet<>();
}
if (!taskConfigUpdates.isEmpty()) {
// Similarly for task config updates
taskConfigUpdatesCopy.set(taskConfigUpdates);
taskConfigUpdates = new HashSet<>();
}
} else {
log.trace("Skipping config updates with incremental cooperative rebalancing "
+ "since no config rebalance is required "
+ "and there are no connector config, task config, or target state changes pending");
}
return retValue;
}
private void processConnectorConfigUpdates(Set<String> connectorConfigUpdates) {
// If we only have connector config updates, we can just bounce the updated connectors that are
// currently assigned to this worker.
Set<String> localConnectors = assignment == null ? Collections.emptySet() : new HashSet<>(assignment.connectors());
Collection<Callable<Void>> connectorsToStart = new ArrayList<>();
log.trace("Processing connector config updates; "
+ "currently-owned connectors are {}, and to-be-updated connectors are {}",
localConnectors,
connectorConfigUpdates);
for (String connectorName : connectorConfigUpdates) {
if (!localConnectors.contains(connectorName)) {
log.trace("Skipping config update for connector {} as it is not owned by this worker",
connectorName);
continue;
}
boolean remains = configState.contains(connectorName);
log.info("Handling connector-only config update by {} connector {}",
remains ? "restarting" : "stopping", connectorName);
worker.stopAndAwaitConnector(connectorName);
// The update may be a deletion, so verify we actually need to restart the connector
if (remains) {
connectorsToStart.add(getConnectorStartingCallable(connectorName));
}
}
startAndStop(connectorsToStart);
}
private void processTargetStateChanges(Set<String> connectorTargetStateChanges) {
log.trace("Processing target state updates; "
+ "currently-known connectors are {}, and to-be-updated connectors are {}",
configState.connectors(), connectorTargetStateChanges);
for (String connector : connectorTargetStateChanges) {
TargetState targetState = configState.targetState(connector);
if (!configState.connectors().contains(connector)) {
log.debug("Received target state change for unknown connector: {}", connector);
continue;
}
// we must propagate the state change to the worker so that the connector's
// tasks can transition to the new target state
worker.setTargetState(connector, targetState, (error, newState) -> {
if (error != null) {
log.error("Failed to transition connector to target state", error);
return;
}
// additionally, if the worker is running the connector itself, then we need to
// request reconfiguration to ensure that config changes while paused take effect
if (newState == TargetState.STARTED) {
requestTaskReconfiguration(connector);
}
});
}
}
private void processTaskConfigUpdatesWithIncrementalCooperative(Set<ConnectorTaskId> taskConfigUpdates) {
Set<ConnectorTaskId> localTasks = assignment == null
? Collections.emptySet()
: new HashSet<>(assignment.tasks());
log.trace("Processing task config updates with incremental cooperative rebalance protocol; "
+ "currently-owned tasks are {}, and to-be-updated tasks are {}",
localTasks, taskConfigUpdates);
Set<String> connectorsWhoseTasksToStop = taskConfigUpdates.stream()
.map(ConnectorTaskId::connector).collect(Collectors.toSet());
stopReconfiguredTasks(connectorsWhoseTasksToStop);
}
private void stopReconfiguredTasks(Set<String> connectors) {
Set<ConnectorTaskId> localTasks = assignment == null
? Collections.emptySet()
: new HashSet<>(assignment.tasks());
List<ConnectorTaskId> tasksToStop = localTasks.stream()
.filter(taskId -> connectors.contains(taskId.connector()))
.collect(Collectors.toList());
if (tasksToStop.isEmpty()) {
// The rest of the method would essentially be a no-op so this isn't strictly necessary,
// but it prevents an unnecessary log message from being emitted
return;
}
log.info("Handling task config update by stopping tasks {}, which will be restarted after rebalance if still assigned to this worker", tasksToStop);
worker.stopAndAwaitTasks(tasksToStop);
tasksToRestart.addAll(tasksToStop);
}
// public for testing
public void halt() {
synchronized (this) {
// Clean up any connectors and tasks that are still running.
log.info("Stopping connectors and tasks that are still assigned to this worker.");
List<Callable<Void>> callables = new ArrayList<>();
for (String connectorName : new ArrayList<>(worker.connectorNames())) {
callables.add(getConnectorStoppingCallable(connectorName));
}
for (ConnectorTaskId taskId : new ArrayList<>(worker.taskIds())) {
callables.add(getTaskStoppingCallable(taskId));
}
startAndStop(callables);
member.stop();
// Explicitly fail any outstanding requests so they actually get a response and get an
// understandable reason for their failure.
DistributedHerderRequest request = requests.pollFirst();
while (request != null) {
request.callback().onCompletion(new ConnectException("Worker is shutting down"), null);
request = requests.pollFirst();
}
stopServices();
}
}
@Override
protected void stopServices() {
try {
super.stopServices();
} finally {
this.uponShutdown.forEach(closeable -> Utils.closeQuietly(closeable, closeable != null ? closeable.toString() : "<unknown>"));
}
}
@Override
public void stop() {
log.info("Herder stopping");
stopping.set(true);
member.wakeup();
herderExecutor.shutdown();
try {
if (!herderExecutor.awaitTermination(workerTasksShutdownTimeoutMs, TimeUnit.MILLISECONDS))
herderExecutor.shutdownNow();
forwardRequestExecutor.shutdown();
startAndStopExecutor.shutdown();
if (!forwardRequestExecutor.awaitTermination(FORWARD_REQUEST_SHUTDOWN_TIMEOUT_MS, TimeUnit.MILLISECONDS))
forwardRequestExecutor.shutdownNow();
if (!startAndStopExecutor.awaitTermination(START_AND_STOP_SHUTDOWN_TIMEOUT_MS, TimeUnit.MILLISECONDS))
startAndStopExecutor.shutdownNow();
} catch (InterruptedException e) {
// ignore
}
log.info("Herder stopped");
running = false;
}
@Override
public void connectors(final Callback<Collection<String>> callback) {
log.trace("Submitting connector listing request");
addRequest(
() -> {
if (!checkRebalanceNeeded(callback))
callback.onCompletion(null, configState.connectors());
return null;
},
forwardErrorCallback(callback)
);
}
@Override
public void connectorInfo(final String connName, final Callback<ConnectorInfo> callback) {
log.trace("Submitting connector info request {}", connName);
addRequest(
() -> {
if (checkRebalanceNeeded(callback))
return null;
if (!configState.contains(connName)) {
callback.onCompletion(
new NotFoundException("Connector " + connName + " not found"), null);
} else {
callback.onCompletion(null, connectorInfo(connName));
}
return null;
},
forwardErrorCallback(callback)
);
}
@Override
public void tasksConfig(String connName, final Callback<Map<ConnectorTaskId, Map<String, String>>> callback) {
log.trace("Submitting tasks config request {}", connName);
addRequest(
() -> {
if (checkRebalanceNeeded(callback))
return null;
if (!configState.contains(connName)) {
callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
} else {
callback.onCompletion(null, buildTasksConfig(connName));
}
return null;
},
forwardErrorCallback(callback)
);
}
@Override
protected Map<String, String> rawConfig(String connName) {
return configState.rawConnectorConfig(connName);
}
@Override
public void connectorConfig(String connName, final Callback<Map<String, String>> callback) {
log.trace("Submitting connector config read request {}", connName);
super.connectorConfig(connName, callback);
}
@Override
public void deleteConnectorConfig(final String connName, final Callback<Created<ConnectorInfo>> callback) {
addRequest(
() -> {
log.trace("Handling connector config request {}", connName);
if (!isLeader()) {
callback.onCompletion(new NotLeaderException("Only the leader can delete connector configs.", leaderUrl()), null);
return null;
}
if (!configState.contains(connName)) {
callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
} else {
log.trace("Removing connector config {} {}", connName, configState.connectors());
writeToConfigTopicAsLeader(() -> configBackingStore.removeConnectorConfig(connName));
callback.onCompletion(null, new Created<>(false, null));
}
return null;
},
forwardErrorCallback(callback)
);
}
@Override
protected Map<String, ConfigValue> validateSinkConnectorConfig(SinkConnector connector, ConfigDef configDef, Map<String, String> config) {
Map<String, ConfigValue> result = super.validateSinkConnectorConfig(connector, configDef, config);
validateSinkConnectorGroupId(result);
return result;
}
@Override
protected Map<String, ConfigValue> validateSourceConnectorConfig(SourceConnector connector, ConfigDef configDef, Map<String, String> config) {
Map<String, ConfigValue> result = super.validateSourceConnectorConfig(connector, configDef, config);
validateSourceConnectorExactlyOnceSupport(config, result, connector);
validateSourceConnectorTransactionBoundary(config, result, connector);
return result;
}
private void validateSinkConnectorGroupId(Map<String, ConfigValue> validatedConfig) {
ConfigValue validatedName = validatedConfig.get(ConnectorConfig.NAME_CONFIG);
String name = (String) validatedName.value();
if (workerGroupId.equals(SinkUtils.consumerGroupId(name))) {
validatedName.addErrorMessage("Consumer group for sink connector named " + name +
" conflicts with Connect worker group " + workerGroupId);
}
}
private void validateSourceConnectorExactlyOnceSupport(
Map<String, String> rawConfig,
Map<String, ConfigValue> validatedConfig,
SourceConnector connector) {
ConfigValue validatedExactlyOnceSupport = validatedConfig.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG);
if (validatedExactlyOnceSupport.errorMessages().isEmpty()) {
// Should be safe to parse the enum from the user-provided value since it's passed validation so far
SourceConnectorConfig.ExactlyOnceSupportLevel exactlyOnceSupportLevel =
SourceConnectorConfig.ExactlyOnceSupportLevel.fromProperty(Objects.toString(validatedExactlyOnceSupport.value()));
if (SourceConnectorConfig.ExactlyOnceSupportLevel.REQUIRED.equals(exactlyOnceSupportLevel)) {
if (!config.exactlyOnceSourceEnabled()) {
validatedExactlyOnceSupport.addErrorMessage("This worker does not have exactly-once source support enabled.");
}
try {
ExactlyOnceSupport exactlyOnceSupport = connector.exactlyOnceSupport(rawConfig);
if (!ExactlyOnceSupport.SUPPORTED.equals(exactlyOnceSupport)) {
final String validationErrorMessage;
// Would do a switch here but that doesn't permit matching on null values
if (exactlyOnceSupport == null) {
validationErrorMessage = "The connector does not implement the API required for preflight validation of exactly-once "
+ "source support. Please consult the documentation for the connector to determine whether it supports exactly-once "
+ "semantics, and then consider reconfiguring the connector to use the value \""
+ SourceConnectorConfig.ExactlyOnceSupportLevel.REQUESTED
+ "\" for this property (which will disable this preflight check and allow the connector to be created).";
} else if (ExactlyOnceSupport.UNSUPPORTED.equals(exactlyOnceSupport)) {
validationErrorMessage = "The connector does not support exactly-once semantics with the provided configuration.";
} else {
throw new ConnectException("Unexpected value returned from SourceConnector::exactlyOnceSupport: " + exactlyOnceSupport);
}
validatedExactlyOnceSupport.addErrorMessage(validationErrorMessage);
}
} catch (Exception e) {
log.error("Failed while validating connector support for exactly-once semantics", e);
String validationErrorMessage = "An unexpected error occurred during validation";
String failureMessage = e.getMessage();
if (failureMessage != null && !failureMessage.trim().isEmpty()) {
validationErrorMessage += ": " + failureMessage.trim();
} else {
validationErrorMessage += "; please see the worker logs for more details.";
}
validatedExactlyOnceSupport.addErrorMessage(validationErrorMessage);
}
}
}
}
private void validateSourceConnectorTransactionBoundary(
Map<String, String> rawConfig,
Map<String, ConfigValue> validatedConfig,
SourceConnector connector) {
ConfigValue validatedTransactionBoundary = validatedConfig.get(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG);
if (validatedTransactionBoundary.errorMessages().isEmpty()) {
// Should be safe to parse the enum from the user-provided value since it's passed validation so far
SourceTask.TransactionBoundary transactionBoundary =
SourceTask.TransactionBoundary.fromProperty(Objects.toString(validatedTransactionBoundary.value()));
if (SourceTask.TransactionBoundary.CONNECTOR.equals(transactionBoundary)) {
try {
ConnectorTransactionBoundaries connectorTransactionSupport = connector.canDefineTransactionBoundaries(rawConfig);
if (connectorTransactionSupport == null) {
validatedTransactionBoundary.addErrorMessage(
"This connector has returned a null value from its canDefineTransactionBoundaries method, which is not permitted. " +
"The connector will be treated as if it cannot define its own transaction boundaries, and cannot be configured with " +
"'" + SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG + "' set to '" + SourceTask.TransactionBoundary.CONNECTOR + "'."
);
} else if (!ConnectorTransactionBoundaries.SUPPORTED.equals(connectorTransactionSupport)) {
validatedTransactionBoundary.addErrorMessage(
"The connector does not support connector-defined transaction boundaries with the given configuration. "
+ "Please reconfigure it to use a different transaction boundary definition.");
}
} catch (Exception e) {
log.error("Failed while validating connector support for defining its own transaction boundaries", e);
String validationErrorMessage = "An unexpected error occurred during validation";
String failureMessage = e.getMessage();
if (failureMessage != null && !failureMessage.trim().isEmpty()) {
validationErrorMessage += ": " + failureMessage.trim();
} else {
validationErrorMessage += "; please see the worker logs for more details.";
}
validatedTransactionBoundary.addErrorMessage(validationErrorMessage);
}
}
}
}