forked from apache/druid
-
Notifications
You must be signed in to change notification settings - Fork 32
/
HttpRemoteTaskRunner.java
1440 lines (1291 loc) · 50 KB
/
HttpRemoteTaskRunner.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.druid.indexing.overlord.hrtr;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.io.ByteSource;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListenableScheduledFuture;
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.curator.framework.CuratorFramework;
import org.apache.druid.concurrent.LifecycleLock;
import org.apache.druid.discovery.DiscoveryDruidNode;
import org.apache.druid.discovery.DruidNodeDiscovery;
import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
import org.apache.druid.discovery.NodeRole;
import org.apache.druid.discovery.WorkerNodeService;
import org.apache.druid.indexer.RunnerTaskState;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.ImmutableWorkerInfo;
import org.apache.druid.indexing.overlord.RemoteTaskRunnerWorkItem;
import org.apache.druid.indexing.overlord.TaskRunnerListener;
import org.apache.druid.indexing.overlord.TaskRunnerUtils;
import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
import org.apache.druid.indexing.overlord.TaskStorage;
import org.apache.druid.indexing.overlord.WorkerTaskRunner;
import org.apache.druid.indexing.overlord.autoscaling.ProvisioningService;
import org.apache.druid.indexing.overlord.autoscaling.ProvisioningStrategy;
import org.apache.druid.indexing.overlord.autoscaling.ScalingStats;
import org.apache.druid.indexing.overlord.config.HttpRemoteTaskRunnerConfig;
import org.apache.druid.indexing.overlord.config.WorkerTaskRunnerConfig;
import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig;
import org.apache.druid.indexing.overlord.setup.WorkerSelectStrategy;
import org.apache.druid.indexing.worker.TaskAnnouncement;
import org.apache.druid.indexing.worker.Worker;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.concurrent.ScheduledExecutors;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.java.util.http.client.Request;
import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler;
import org.apache.druid.server.initialization.IndexerZkConfig;
import org.apache.druid.tasklogs.TaskLogStreamer;
import org.apache.zookeeper.KeeperException;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.InputStream;
import java.net.URL;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
/**
* A Remote TaskRunner to manage tasks on Middle Manager nodes using internal-discovery({@link DruidNodeDiscoveryProvider})
* to discover them and Http.
* Middle Managers expose 3 HTTP endpoints
* 1. POST request for assigning a task
* 2. POST request for shutting down a task
* 3. GET request for getting list of assigned, running, completed tasks on Middle Manager and its enable/disable status.
* This endpoint is implemented to support long poll and holds the request till there is a change. This class
* sends the next request immediately as the previous finishes to keep the state up-to-date.
* <p>
* ZK_CLEANUP_TODO : As of 0.11.1, it is required to cleanup task status paths from ZK which are created by the
* workers to support deprecated RemoteTaskRunner. So a method "scheduleCompletedTaskStatusCleanupFromZk()" is added'
* which should be removed in the release that removes RemoteTaskRunner legacy ZK updation WorkerTaskMonitor class.
*/
public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
{
private static final EmittingLogger log = new EmittingLogger(HttpRemoteTaskRunner.class);
private final LifecycleLock lifecycleLock = new LifecycleLock();
// Executor for assigning pending tasks to workers.
private final ExecutorService pendingTasksExec;
// All known tasks
private final ConcurrentMap<String, HttpRemoteTaskRunnerWorkItem> tasks = new ConcurrentHashMap<>();
// All discovered workers.
private final ConcurrentMap<String, WorkerHolder> workers = new ConcurrentHashMap<>();
// Executor for syncing state of each worker.
private final ScheduledExecutorService workersSyncExec;
// Workers that have been marked as lazy. these workers are not running any tasks and can be terminated safely by the scaling policy.
private final ConcurrentMap<String, WorkerHolder> lazyWorkers = new ConcurrentHashMap<>();
// Workers that have been blacklisted.
private final ConcurrentHashMap<String, WorkerHolder> blackListedWorkers = new ConcurrentHashMap<>();
// workers which were assigned a task and are yet to acknowledge same.
// Map: workerId -> taskId
private final ConcurrentMap<String, String> workersWithUnacknowledgedTask = new ConcurrentHashMap<>();
// Executor to complete cleanup of workers which have disappeared.
private final ListeningScheduledExecutorService cleanupExec;
private final ConcurrentMap<String, ScheduledFuture> removedWorkerCleanups = new ConcurrentHashMap<>();
// Guards the pending/running/complete lists of tasks and list of workers
// statusLock.notifyAll() is called whenever there is a possibility of worker slot to run task becoming available.
// statusLock.notifyAll() is called whenever a task status or location changes.
private final Object statusLock = new Object();
// task runner listeners
private final CopyOnWriteArrayList<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
private final ProvisioningStrategy<WorkerTaskRunner> provisioningStrategy;
private ProvisioningService provisioningService;
private final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider;
private final HttpClient httpClient;
private final ObjectMapper smileMapper;
private final Supplier<WorkerBehaviorConfig> workerConfigRef;
private final HttpRemoteTaskRunnerConfig config;
private final TaskStorage taskStorage;
// ZK_CLEANUP_TODO : Remove these when RemoteTaskRunner and WorkerTaskMonitor are removed.
private static final Joiner JOINER = Joiner.on("/");
private final CuratorFramework cf;
private final ScheduledExecutorService zkCleanupExec;
private final IndexerZkConfig indexerZkConfig;
public HttpRemoteTaskRunner(
ObjectMapper smileMapper,
HttpRemoteTaskRunnerConfig config,
HttpClient httpClient,
Supplier<WorkerBehaviorConfig> workerConfigRef,
ProvisioningStrategy<WorkerTaskRunner> provisioningStrategy,
DruidNodeDiscoveryProvider druidNodeDiscoveryProvider,
TaskStorage taskStorage,
CuratorFramework cf,
IndexerZkConfig indexerZkConfig
)
{
this.smileMapper = smileMapper;
this.config = config;
this.httpClient = httpClient;
this.druidNodeDiscoveryProvider = druidNodeDiscoveryProvider;
this.taskStorage = taskStorage;
this.workerConfigRef = workerConfigRef;
this.pendingTasksExec = Execs.multiThreaded(
config.getPendingTasksRunnerNumThreads(),
"hrtr-pending-tasks-runner-%d"
);
this.workersSyncExec = ScheduledExecutors.fixed(
config.getWorkerSyncNumThreads(),
"HttpRemoteTaskRunner-worker-sync-%d"
);
this.cleanupExec = MoreExecutors.listeningDecorator(
ScheduledExecutors.fixed(1, "HttpRemoteTaskRunner-Worker-Cleanup-%d")
);
this.cf = cf;
this.indexerZkConfig = indexerZkConfig;
this.zkCleanupExec = ScheduledExecutors.fixed(
1,
"HttpRemoteTaskRunner-zk-cleanup-%d"
);
this.provisioningStrategy = provisioningStrategy;
}
@Override
@LifecycleStart
public void start()
{
if (!lifecycleLock.canStart()) {
return;
}
try {
log.info("Starting...");
scheduleCompletedTaskStatusCleanupFromZk();
startWorkersHandling();
ScheduledExecutors.scheduleAtFixedRate(
cleanupExec,
Period.ZERO.toStandardDuration(),
config.getWorkerBlackListCleanupPeriod().toStandardDuration(),
() -> checkAndRemoveWorkersFromBlackList()
);
provisioningService = provisioningStrategy.makeProvisioningService(this);
scheduleSyncMonitoring();
lifecycleLock.started();
log.info("Started.");
}
catch (Exception e) {
throw new RuntimeException(e);
}
finally {
lifecycleLock.exitStart();
}
}
private void scheduleCompletedTaskStatusCleanupFromZk()
{
zkCleanupExec.scheduleAtFixedRate(
() -> {
try {
List<String> workers;
try {
workers = cf.getChildren().forPath(indexerZkConfig.getStatusPath());
}
catch (KeeperException.NoNodeException e) {
// statusPath doesn't exist yet; can occur if no middleManagers have started.
workers = ImmutableList.of();
}
Set<String> knownActiveTaskIds = new HashSet<>();
if (!workers.isEmpty()) {
for (Task task : taskStorage.getActiveTasks()) {
knownActiveTaskIds.add(task.getId());
}
}
for (String workerId : workers) {
String workerStatusPath = JOINER.join(indexerZkConfig.getStatusPath(), workerId);
List<String> taskIds;
try {
taskIds = cf.getChildren().forPath(workerStatusPath);
}
catch (KeeperException.NoNodeException e) {
taskIds = ImmutableList.of();
}
for (String taskId : taskIds) {
if (!knownActiveTaskIds.contains(taskId)) {
String taskStatusPath = JOINER.join(workerStatusPath, taskId);
try {
cf.delete().guaranteed().forPath(taskStatusPath);
}
catch (KeeperException.NoNodeException e) {
log.info("Failed to delete taskStatusPath[%s].", taskStatusPath);
}
}
}
}
}
catch (InterruptedException ex) {
Thread.currentThread().interrupt();
}
catch (Exception ex) {
log.error(ex, "Unknown error while doing task status cleanup in ZK.");
}
},
1,
5,
TimeUnit.MINUTES
);
}
private ImmutableWorkerInfo findWorkerToRunTask(Task task)
{
WorkerBehaviorConfig workerConfig = workerConfigRef.get();
WorkerSelectStrategy strategy;
if (workerConfig == null || workerConfig.getSelectStrategy() == null) {
strategy = WorkerBehaviorConfig.DEFAULT_STRATEGY;
log.debug("No worker selection strategy set. Using default of [%s]", strategy.getClass().getSimpleName());
} else {
strategy = workerConfig.getSelectStrategy();
}
return strategy.findWorkerForTask(
config,
ImmutableMap.copyOf(
Maps.transformEntries(
Maps.filterEntries(
workers,
new Predicate<Map.Entry<String, WorkerHolder>>()
{
@Override
public boolean apply(Map.Entry<String, WorkerHolder> input)
{
return !lazyWorkers.containsKey(input.getKey()) &&
!workersWithUnacknowledgedTask.containsKey(input.getKey()) &&
!blackListedWorkers.containsKey(input.getKey());
}
}
),
(String key, WorkerHolder value) -> value.toImmutable()
)
),
task
);
}
private boolean runTaskOnWorker(
final HttpRemoteTaskRunnerWorkItem workItem,
final String workerHost
) throws Exception
{
String taskId = workItem.getTaskId();
WorkerHolder workerHolder = workers.get(workerHost);
if (workerHolder == null || lazyWorkers.containsKey(workerHost) || blackListedWorkers.containsKey(workerHost)) {
log.info("Not assigning task[%s] to removed or marked lazy/blacklisted worker[%s]", taskId, workerHost);
return false;
}
log.info("Asking Worker[%s] to run task[%s]", workerHost, taskId);
if (workerHolder.assignTask(workItem.getTask())) {
// Don't assign new tasks until the task we just assigned is actually running
// on a worker - this avoids overflowing a worker with tasks
long waitMs = config.getTaskAssignmentTimeout().toStandardDuration().getMillis();
long waitStart = System.currentTimeMillis();
boolean isTaskAssignmentTimedOut = false;
synchronized (statusLock) {
while (tasks.containsKey(taskId)
&& tasks.get(taskId).getState() == HttpRemoteTaskRunnerWorkItem.State.PENDING) {
long remaining = waitMs - (System.currentTimeMillis() - waitStart);
if (remaining > 0) {
statusLock.wait(remaining);
} else {
isTaskAssignmentTimedOut = true;
break;
}
}
}
if (isTaskAssignmentTimedOut) {
log.makeAlert(
"Task assignment timed out on worker [%s], never ran task [%s] in timeout[%s]!",
workerHost,
taskId,
config.getTaskAssignmentTimeout()
).emit();
taskComplete(workItem, workerHolder, TaskStatus.failure(taskId));
}
return true;
} else {
return false;
}
}
// CAUTION: This method calls RemoteTaskRunnerWorkItem.setResult(..) which results in TaskQueue.notifyStatus() being called
// because that is attached by TaskQueue to task result future. So, this method must not be called with "statusLock"
// held. See https://github.com/apache/incubator-druid/issues/6201
private void taskComplete(
HttpRemoteTaskRunnerWorkItem taskRunnerWorkItem,
WorkerHolder workerHolder,
TaskStatus taskStatus
)
{
Preconditions.checkState(!Thread.holdsLock(statusLock), "Current thread must not hold statusLock.");
Preconditions.checkNotNull(taskRunnerWorkItem, "taskRunnerWorkItem");
Preconditions.checkNotNull(taskStatus, "taskStatus");
if (workerHolder != null) {
log.info(
"Worker[%s] completed task[%s] with status[%s]",
workerHolder.getWorker().getHost(),
taskStatus.getId(),
taskStatus.getStatusCode()
);
// Worker is done with this task
workerHolder.setLastCompletedTaskTime(DateTimes.nowUtc());
}
// Notify interested parties
taskRunnerWorkItem.setResult(taskStatus);
TaskRunnerUtils.notifyStatusChanged(listeners, taskStatus.getId(), taskStatus);
// Update success/failure counters, Blacklist node if there are too many failures.
if (workerHolder != null) {
blacklistWorkerIfNeeded(taskStatus, workerHolder);
}
synchronized (statusLock) {
statusLock.notifyAll();
}
}
private void startWorkersHandling() throws InterruptedException
{
final CountDownLatch workerViewInitialized = new CountDownLatch(1);
DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.MIDDLE_MANAGER);
druidNodeDiscovery.registerListener(
new DruidNodeDiscovery.Listener()
{
@Override
public void nodesAdded(Collection<DiscoveryDruidNode> nodes)
{
nodes.forEach(node -> addWorker(toWorker(node)));
}
@Override
public void nodesRemoved(Collection<DiscoveryDruidNode> nodes)
{
nodes.forEach(node -> removeWorker(toWorker(node)));
}
@Override
public void nodeViewInitialized()
{
//CountDownLatch.countDown() does nothing when count has already reached 0.
workerViewInitialized.countDown();
}
}
);
long workerDiscoveryStartTime = System.currentTimeMillis();
while (!workerViewInitialized.await(30, TimeUnit.SECONDS)) {
if (System.currentTimeMillis() - workerDiscoveryStartTime > TimeUnit.MINUTES.toMillis(5)) {
throw new ISE("WTF! Couldn't discover workers.");
} else {
log.info("Waiting for worker discovery...");
}
}
log.info("[%s] Workers are discovered.", workers.size());
// Wait till all worker state is sync'd so that we know which worker is running/completed what tasks or else
// We would start assigning tasks which are pretty soon going to be reported by discovered workers.
for (WorkerHolder worker : workers.values()) {
log.info("Waiting for worker[%s] to sync state...", worker.getWorker().getHost());
worker.waitForInitialization();
}
log.info("Workers have sync'd state successfully.");
}
private Worker toWorker(DiscoveryDruidNode node)
{
return new Worker(
node.getDruidNode().getServiceScheme(),
node.getDruidNode().getHostAndPortToUse(),
((WorkerNodeService) node.getServices().get(WorkerNodeService.DISCOVERY_SERVICE_KEY)).getIp(),
((WorkerNodeService) node.getServices().get(WorkerNodeService.DISCOVERY_SERVICE_KEY)).getCapacity(),
((WorkerNodeService) node.getServices().get(WorkerNodeService.DISCOVERY_SERVICE_KEY)).getVersion()
);
}
private void addWorker(final Worker worker)
{
synchronized (workers) {
log.info("Worker[%s] reportin' for duty!", worker.getHost());
cancelWorkerCleanup(worker.getHost());
WorkerHolder holder = workers.get(worker.getHost());
if (holder == null) {
holder = createWorkerHolder(smileMapper, httpClient, config, workersSyncExec, this::taskAddedOrUpdated, worker);
holder.start();
workers.put(worker.getHost(), holder);
} else {
log.info("Worker[%s] already exists.", worker.getHost());
}
}
synchronized (statusLock) {
statusLock.notifyAll();
}
}
protected WorkerHolder createWorkerHolder(
ObjectMapper smileMapper,
HttpClient httpClient,
HttpRemoteTaskRunnerConfig config,
ScheduledExecutorService workersSyncExec,
WorkerHolder.Listener listener,
Worker worker
)
{
return new WorkerHolder(smileMapper, httpClient, config, workersSyncExec, listener, worker);
}
private void removeWorker(final Worker worker)
{
synchronized (workers) {
log.info("Kaboom! Worker[%s] removed!", worker.getHost());
WorkerHolder workerHolder = workers.remove(worker.getHost());
if (workerHolder != null) {
try {
workerHolder.stop();
scheduleTasksCleanupForWorker(worker.getHost());
}
catch (Exception e) {
throw new RuntimeException(e);
}
finally {
checkAndRemoveWorkersFromBlackList();
}
}
lazyWorkers.remove(worker.getHost());
}
}
private boolean cancelWorkerCleanup(String workerHost)
{
ScheduledFuture previousCleanup = removedWorkerCleanups.remove(workerHost);
if (previousCleanup != null) {
log.info("Cancelling Worker[%s] scheduled task cleanup", workerHost);
previousCleanup.cancel(false);
}
return previousCleanup != null;
}
private void scheduleTasksCleanupForWorker(final String workerHostAndPort)
{
cancelWorkerCleanup(workerHostAndPort);
final ListenableScheduledFuture<?> cleanupTask = cleanupExec.schedule(
new Runnable()
{
@Override
public void run()
{
log.info("Running scheduled cleanup for Worker[%s]", workerHostAndPort);
try {
Set<HttpRemoteTaskRunnerWorkItem> tasksToFail = new HashSet<>();
synchronized (statusLock) {
for (Map.Entry<String, HttpRemoteTaskRunnerWorkItem> e : tasks.entrySet()) {
if (e.getValue().getState() == HttpRemoteTaskRunnerWorkItem.State.RUNNING) {
Worker w = e.getValue().getWorker();
if (w != null && w.getHost().equals(workerHostAndPort)) {
tasksToFail.add(e.getValue());
}
}
}
}
for (HttpRemoteTaskRunnerWorkItem taskItem : tasksToFail) {
if (!taskItem.getResult().isDone()) {
log.info(
"Failing task[%s] because worker[%s] disappeared and did not report within cleanup timeout[%s].",
workerHostAndPort,
taskItem.getTaskId(),
config.getTaskCleanupTimeout()
);
taskComplete(taskItem, null, TaskStatus.failure(taskItem.getTaskId()));
}
}
}
catch (Exception e) {
log.makeAlert("Exception while cleaning up worker[%s]", workerHostAndPort).emit();
throw new RuntimeException(e);
}
}
},
config.getTaskCleanupTimeout().toStandardDuration().getMillis(),
TimeUnit.MILLISECONDS
);
removedWorkerCleanups.put(workerHostAndPort, cleanupTask);
// Remove this entry from removedWorkerCleanups when done, if it's actually the one in there.
Futures.addCallback(
cleanupTask,
new FutureCallback<Object>()
{
@Override
public void onSuccess(Object result)
{
removedWorkerCleanups.remove(workerHostAndPort, cleanupTask);
}
@Override
public void onFailure(Throwable t)
{
removedWorkerCleanups.remove(workerHostAndPort, cleanupTask);
}
}
);
}
private void scheduleSyncMonitoring()
{
workersSyncExec.scheduleAtFixedRate(
() -> {
log.debug("Running the Sync Monitoring.");
try {
for (Map.Entry<String, WorkerHolder> e : workers.entrySet()) {
WorkerHolder workerHolder = e.getValue();
if (!workerHolder.getUnderlyingSyncer().isOK()) {
synchronized (workers) {
// check again that server is still there and only then reset.
if (workers.containsKey(e.getKey())) {
log.makeAlert(
"Worker[%s] is not syncing properly. Current state is [%s]. Resetting it.",
workerHolder.getWorker().getHost(),
workerHolder.getUnderlyingSyncer().getDebugInfo()
).emit();
removeWorker(workerHolder.getWorker());
addWorker(workerHolder.getWorker());
}
}
}
}
}
catch (Exception ex) {
if (ex instanceof InterruptedException) {
Thread.currentThread().interrupt();
} else {
log.makeAlert(ex, "Exception in sync monitoring.").emit();
}
}
},
1,
5,
TimeUnit.MINUTES
);
}
/**
* This method returns the debugging information exposed by {@link HttpRemoteTaskRunnerResource} and meant
* for that use only. It must not be used for any other purpose.
*/
public Map<String, Object> getDebugInfo()
{
Preconditions.checkArgument(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS));
Map<String, Object> result = new HashMap<>(workers.size());
for (Map.Entry<String, WorkerHolder> e : workers.entrySet()) {
WorkerHolder serverHolder = e.getValue();
result.put(
e.getKey(),
serverHolder.getUnderlyingSyncer().getDebugInfo()
);
}
return result;
}
private void checkAndRemoveWorkersFromBlackList()
{
boolean shouldRunPendingTasks = false;
Iterator<Map.Entry<String, WorkerHolder>> iterator = blackListedWorkers.entrySet().iterator();
while (iterator.hasNext()) {
Map.Entry<String, WorkerHolder> e = iterator.next();
if (shouldRemoveNodeFromBlackList(e.getValue())) {
iterator.remove();
e.getValue().resetContinuouslyFailedTasksCount();
e.getValue().setBlacklistedUntil(null);
shouldRunPendingTasks = true;
}
}
if (shouldRunPendingTasks) {
synchronized (statusLock) {
statusLock.notifyAll();
}
}
}
private boolean shouldRemoveNodeFromBlackList(WorkerHolder workerHolder)
{
if (!workers.containsKey(workerHolder.getWorker().getHost())) {
return true;
}
if (blackListedWorkers.size() > workers.size() * (config.getMaxPercentageBlacklistWorkers() / 100.0)) {
log.info(
"Removing [%s] from blacklist because percentage of blacklisted workers exceeds [%d]",
workerHolder.getWorker(),
config.getMaxPercentageBlacklistWorkers()
);
return true;
}
long remainingMillis = workerHolder.getBlacklistedUntil().getMillis() - System.currentTimeMillis();
if (remainingMillis <= 0) {
log.info("Removing [%s] from blacklist because backoff time elapsed", workerHolder.getWorker());
return true;
}
log.info("[%s] still blacklisted for [%,ds]", workerHolder.getWorker(), remainingMillis / 1000);
return false;
}
private void blacklistWorkerIfNeeded(TaskStatus taskStatus, WorkerHolder workerHolder)
{
synchronized (blackListedWorkers) {
if (taskStatus.isSuccess()) {
workerHolder.resetContinuouslyFailedTasksCount();
if (blackListedWorkers.remove(workerHolder.getWorker().getHost()) != null) {
workerHolder.setBlacklistedUntil(null);
log.info("[%s] removed from blacklist because a task finished with SUCCESS", workerHolder.getWorker());
}
} else if (taskStatus.isFailure()) {
workerHolder.incrementContinuouslyFailedTasksCount();
}
if (workerHolder.getContinuouslyFailedTasksCount() > config.getMaxRetriesBeforeBlacklist() &&
blackListedWorkers.size() <= workers.size() * (config.getMaxPercentageBlacklistWorkers() / 100.0) - 1) {
workerHolder.setBlacklistedUntil(DateTimes.nowUtc().plus(config.getWorkerBlackListBackoffTime()));
if (blackListedWorkers.put(workerHolder.getWorker().getHost(), workerHolder) == null) {
log.info(
"Blacklisting [%s] until [%s] after [%,d] failed tasks in a row.",
workerHolder.getWorker(),
workerHolder.getBlacklistedUntil(),
workerHolder.getContinuouslyFailedTasksCount()
);
}
}
}
}
@Override
public Collection<ImmutableWorkerInfo> getWorkers()
{
return workers.values().stream().map(worker -> worker.toImmutable()).collect(Collectors.toList());
}
@Override
public Collection<Worker> getLazyWorkers()
{
return lazyWorkers.values().stream().map(holder -> holder.getWorker()).collect(Collectors.toList());
}
@Override
public Collection<Worker> markWorkersLazy(Predicate<ImmutableWorkerInfo> isLazyWorker, int maxWorkers)
{
synchronized (statusLock) {
Iterator<String> iterator = workers.keySet().iterator();
while (iterator.hasNext()) {
String worker = iterator.next();
WorkerHolder workerHolder = workers.get(worker);
try {
if (isWorkerOkForMarkingLazy(workerHolder.getWorker()) && isLazyWorker.apply(workerHolder.toImmutable())) {
log.info("Adding Worker[%s] to lazySet!", workerHolder.getWorker().getHost());
lazyWorkers.put(worker, workerHolder);
if (lazyWorkers.size() == maxWorkers) {
// only mark excess workers as lazy and allow their cleanup
break;
}
}
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
return getLazyWorkers();
}
}
private boolean isWorkerOkForMarkingLazy(Worker worker)
{
// Check that worker is not running any tasks and no task is being assigned to it.
synchronized (statusLock) {
if (workersWithUnacknowledgedTask.containsKey(worker.getHost())) {
return false;
}
for (Map.Entry<String, HttpRemoteTaskRunnerWorkItem> e : tasks.entrySet()) {
if (e.getValue().getState() == HttpRemoteTaskRunnerWorkItem.State.RUNNING) {
Worker w = e.getValue().getWorker();
if (w != null && w.getHost().equals(worker.getHost())) {
return false;
}
}
}
return true;
}
}
@Override
public WorkerTaskRunnerConfig getConfig()
{
return config;
}
@Override
public Collection<Task> getPendingTaskPayloads()
{
synchronized (statusLock) {
return tasks.values()
.stream()
.filter(item -> item.getState() == HttpRemoteTaskRunnerWorkItem.State.PENDING)
.map(item -> item.getTask())
.collect(Collectors.toList());
}
}
@Override
public Optional<ByteSource> streamTaskLog(String taskId, long offset)
{
HttpRemoteTaskRunnerWorkItem taskRunnerWorkItem = tasks.get(taskId);
Worker worker = null;
if (taskRunnerWorkItem != null && taskRunnerWorkItem.getState() != HttpRemoteTaskRunnerWorkItem.State.COMPLETE) {
worker = taskRunnerWorkItem.getWorker();
}
if (worker == null || !workers.containsKey(worker.getHost())) {
// Worker is not running this task, it might be available in deep storage
return Optional.absent();
} else {
// Worker is still running this task
final URL url = TaskRunnerUtils.makeWorkerURL(
worker,
"/druid/worker/v1/task/%s/log?offset=%s",
taskId,
Long.toString(offset)
);
return Optional.of(
new ByteSource()
{
@Override
public InputStream openStream() throws IOException
{
try {
return httpClient.go(
new Request(HttpMethod.GET, url),
new InputStreamResponseHandler()
).get();
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
catch (ExecutionException e) {
// Unwrap if possible
Throwables.propagateIfPossible(e.getCause(), IOException.class);
throw new RuntimeException(e);
}
}
}
);
}
}
@Override
public List<Pair<Task, ListenableFuture<TaskStatus>>> restore()
{
return ImmutableList.of();
}
@Override
public void registerListener(TaskRunnerListener listener, Executor executor)
{
for (Pair<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listener.getListenerId())) {
throw new ISE("Listener [%s] already registered", listener.getListenerId());
}
}
final Pair<TaskRunnerListener, Executor> listenerPair = Pair.of(listener, executor);
synchronized (statusLock) {
for (Map.Entry<String, HttpRemoteTaskRunnerWorkItem> entry : tasks.entrySet()) {
if (entry.getValue().getState() == HttpRemoteTaskRunnerWorkItem.State.RUNNING) {
TaskRunnerUtils.notifyLocationChanged(
ImmutableList.of(listenerPair),
entry.getKey(),
entry.getValue().getLocation()
);
}
}
log.info("Registered listener [%s]", listener.getListenerId());
listeners.add(listenerPair);
}
}
@Override
public void unregisterListener(String listenerId)
{
for (Pair<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listenerId)) {
listeners.remove(pair);
log.info("Unregistered listener [%s]", listenerId);
return;
}
}
}
@Override
public ListenableFuture<TaskStatus> run(Task task)
{
Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS), "not started");
synchronized (statusLock) {
HttpRemoteTaskRunnerWorkItem existing = tasks.get(task.getId());
if (existing != null) {
log.info("Assigned a task[%s] that is known already. Ignored.", task.getId());
if (existing.getTask() == null) {
// in case it was discovered from a worker on start() and TaskAnnouncement does not have Task instance
// in it.
existing.setTask(task);
}
return existing.getResult();
} else {
log.info("Adding pending task[%s].", task.getId());
HttpRemoteTaskRunnerWorkItem taskRunnerWorkItem = new HttpRemoteTaskRunnerWorkItem(
task.getId(),
null,
null,
task,
task.getType(),
HttpRemoteTaskRunnerWorkItem.State.PENDING
);
tasks.put(task.getId(), taskRunnerWorkItem);
addPendingTaskToExecutor(task.getId());
return taskRunnerWorkItem.getResult();
}
}
}
private void addPendingTaskToExecutor(final String taskId)
{
pendingTasksExec.execute(
() -> {
while (!Thread.interrupted() && lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) {
ImmutableWorkerInfo immutableWorker;
HttpRemoteTaskRunnerWorkItem taskItem = null;
try {
synchronized (statusLock) {
taskItem = tasks.get(taskId);
if (taskItem == null) {
log.info(
"Task[%s] work item not found. Probably user asked to shutdown before. Not assigning.",
taskId
);
return;
}
if (taskItem.getState() != HttpRemoteTaskRunnerWorkItem.State.PENDING) {
log.info(
"Task[%s] is in state[%s]. Probably some worker already reported it. Not assigning.",
taskId,
taskItem.getState()
);
return;
}
if (taskItem.getTask() == null) {
throw new ISE("WTF! couldn't find Task instance for taskId[%s].", taskId);
}
immutableWorker = findWorkerToRunTask(taskItem.getTask());
if (immutableWorker == null) {
// no free worker, wait for some worker to become free
statusLock.wait(config.getWaitForWorkerSlot().toStandardDuration().getMillis());