/
SeekableStreamSupervisor.java
3306 lines (2967 loc) · 138 KB
/
SeekableStreamSupervisor.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.seekablestream.supervisor;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.MapperFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
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.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.IndexTaskClient;
import org.apache.druid.indexing.common.TaskInfoProvider;
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import org.apache.druid.indexing.overlord.TaskMaster;
import org.apache.druid.indexing.overlord.TaskQueue;
import org.apache.druid.indexing.overlord.TaskRunner;
import org.apache.druid.indexing.overlord.TaskRunnerListener;
import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
import org.apache.druid.indexing.overlord.TaskStorage;
import org.apache.druid.indexing.overlord.supervisor.Supervisor;
import org.apache.druid.indexing.overlord.supervisor.SupervisorReport;
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
import org.apache.druid.indexing.seekablestream.common.StreamException;
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.RetryUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.metadata.EntryExistsException;
import org.apache.druid.segment.indexing.DataSchema;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
import javax.validation.constraints.NotNull;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* this class is the parent class of both the Kafka and Kinesis supervisor. All the main run loop
* logic are similar enough so they're grouped together into this class.
* <p>
* Supervisor responsible for managing the SeekableStreamIndexTasks (Kafka/Kinesis) for a single dataSource. At a high level, the class accepts a
* {@link SeekableStreamSupervisorSpec} which includes the stream name (topic / stream) and configuration as well as an ingestion spec which will
* be used to generate the indexing tasks. The run loop periodically refreshes its view of the stream's partitions
* and the list of running indexing tasks and ensures that all partitions are being read from and that there are enough
* tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of
* stream sequences.
* <p>
*
* @param <PartitionIdType> the type of the partition id, for example, partitions in Kafka are int type while partitions in Kinesis are String type
* @param <SequenceOffsetType> the type of the sequence number or offsets, for example, Kafka uses long offsets while Kinesis uses String sequence numbers
*/
public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetType> implements Supervisor
{
public static final String CHECKPOINTS_CTX_KEY = "checkpoints";
private static final long MAX_RUN_FREQUENCY_MILLIS = 1000;
private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120;
private static final int MAX_INITIALIZATION_RETRIES = 20;
private static final EmittingLogger log = new EmittingLogger(SeekableStreamSupervisor.class);
// Internal data structures
// --------------------------------------------------------
/**
* A TaskGroup is the main data structure used by SeekableStreamSupervisor to organize and monitor stream partitions and
* indexing tasks. All the tasks in a TaskGroup should always be doing the same thing (reading the same partitions and
* starting from the same sequences) and if [replicas] is configured to be 1, a TaskGroup will contain a single task (the
* exception being if the supervisor started up and discovered and adopted some already running tasks). At any given
* time, there should only be up to a maximum of [taskCount] actively-reading task groups (tracked in the [activelyReadingTaskGroups]
* map) + zero or more pending-completion task groups (tracked in [pendingCompletionTaskGroups]).
*/
private class TaskGroup
{
final int groupId;
// This specifies the partitions and starting sequences for this task group. It is set on group creation from the data
// in [partitionGroups] and never changes during the lifetime of this task group, which will live until a task in
// this task group has completed successfully, at which point this will be destroyed and a new task group will be
// created with new starting sequences. This allows us to create replacement tasks for failed tasks that process the
// same sequences, even if the values in [partitionGroups] has been changed.
final ImmutableMap<PartitionIdType, SequenceOffsetType> startingSequences;
// We don't include closed partitions in the starting offsets. However, we keep the full unfiltered map of
// partitions, only used for generating the sequence name, to avoid ambiguity in sequence names if mulitple
// task groups have nothing but closed partitions in their assignments.
final ImmutableMap<PartitionIdType, SequenceOffsetType> unfilteredStartingSequencesForSequenceName;
final ConcurrentHashMap<String, TaskData> tasks = new ConcurrentHashMap<>();
final Optional<DateTime> minimumMessageTime;
final Optional<DateTime> maximumMessageTime;
final Set<PartitionIdType> exclusiveStartSequenceNumberPartitions;
final TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>> checkpointSequences = new TreeMap<>();
final String baseSequenceName;
DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action
TaskGroup(
int groupId,
ImmutableMap<PartitionIdType, SequenceOffsetType> startingSequences,
@Nullable ImmutableMap<PartitionIdType, SequenceOffsetType> unfilteredStartingSequencesForSequenceName,
Optional<DateTime> minimumMessageTime,
Optional<DateTime> maximumMessageTime,
@Nullable Set<PartitionIdType> exclusiveStartSequenceNumberPartitions
)
{
this(
groupId,
startingSequences,
unfilteredStartingSequencesForSequenceName,
minimumMessageTime,
maximumMessageTime,
exclusiveStartSequenceNumberPartitions,
generateSequenceName(
unfilteredStartingSequencesForSequenceName == null
? startingSequences
: unfilteredStartingSequencesForSequenceName,
minimumMessageTime,
maximumMessageTime,
spec.getDataSchema(),
taskTuningConfig
)
);
}
TaskGroup(
int groupId,
ImmutableMap<PartitionIdType, SequenceOffsetType> startingSequences,
@Nullable ImmutableMap<PartitionIdType, SequenceOffsetType> unfilteredStartingSequencesForSequenceName,
Optional<DateTime> minimumMessageTime,
Optional<DateTime> maximumMessageTime,
Set<PartitionIdType> exclusiveStartSequenceNumberPartitions,
String baseSequenceName
)
{
this.groupId = groupId;
this.startingSequences = startingSequences;
this.unfilteredStartingSequencesForSequenceName = unfilteredStartingSequencesForSequenceName == null
? startingSequences
: unfilteredStartingSequencesForSequenceName;
this.minimumMessageTime = minimumMessageTime;
this.maximumMessageTime = maximumMessageTime;
this.checkpointSequences.put(0, startingSequences);
this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null
? exclusiveStartSequenceNumberPartitions
: Collections.emptySet();
this.baseSequenceName = baseSequenceName;
}
int addNewCheckpoint(Map<PartitionIdType, SequenceOffsetType> checkpoint)
{
checkpointSequences.put(checkpointSequences.lastKey() + 1, checkpoint);
return checkpointSequences.lastKey();
}
Set<String> taskIds()
{
return tasks.keySet();
}
}
private class TaskData
{
volatile TaskStatus status;
volatile DateTime startTime;
volatile Map<PartitionIdType, SequenceOffsetType> currentSequences = new HashMap<>();
@Override
public String toString()
{
return "TaskData{" +
"status=" + status +
", startTime=" + startTime +
", checkpointSequences=" + currentSequences +
'}';
}
}
/**
* Notice is used to queue tasks that are internal to the supervisor
*/
private interface Notice
{
void handle() throws ExecutionException, InterruptedException, TimeoutException;
}
private static class StatsFromTaskResult
{
private final String groupId;
private final String taskId;
private final Map<String, Object> stats;
public StatsFromTaskResult(
int groupId,
String taskId,
Map<String, Object> stats
)
{
this.groupId = String.valueOf(groupId);
this.taskId = taskId;
this.stats = stats;
}
public String getGroupId()
{
return groupId;
}
public String getTaskId()
{
return taskId;
}
public Map<String, Object> getStats()
{
return stats;
}
}
private class RunNotice implements Notice
{
@Override
public void handle()
{
long nowTime = System.currentTimeMillis();
if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) {
return;
}
lastRunTime = nowTime;
runInternal();
}
}
private class GracefulShutdownNotice extends ShutdownNotice
{
@Override
public void handle() throws InterruptedException, ExecutionException, TimeoutException
{
gracefulShutdownInternal();
super.handle();
}
}
private class ShutdownNotice implements Notice
{
@Override
public void handle() throws InterruptedException, ExecutionException, TimeoutException
{
recordSupplier.close();
synchronized (stopLock) {
stopped = true;
stopLock.notifyAll();
}
}
}
private class ResetNotice implements Notice
{
final DataSourceMetadata dataSourceMetadata;
ResetNotice(DataSourceMetadata dataSourceMetadata)
{
this.dataSourceMetadata = dataSourceMetadata;
}
@Override
public void handle()
{
resetInternal(dataSourceMetadata);
}
}
protected class CheckpointNotice implements Notice
{
private final int taskGroupId;
private final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> checkpointMetadata;
CheckpointNotice(
int taskGroupId,
SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> checkpointMetadata
)
{
this.taskGroupId = taskGroupId;
this.checkpointMetadata = checkpointMetadata;
}
@Override
public void handle() throws ExecutionException, InterruptedException
{
// check for consistency
// if already received request for this sequenceName and dataSourceMetadata combination then return
final TaskGroup taskGroup = activelyReadingTaskGroups.get(taskGroupId);
if (isValidTaskGroup(taskGroupId, taskGroup)) {
final TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>> checkpoints = taskGroup.checkpointSequences;
// check validity of previousCheckpoint
int index = checkpoints.size();
for (int sequenceId : checkpoints.descendingKeySet()) {
Map<PartitionIdType, SequenceOffsetType> checkpoint = checkpoints.get(sequenceId);
// We have already verified the stream of the current checkpoint is same with that in ioConfig.
// See checkpoint().
if (checkpoint.equals(checkpointMetadata.getSeekableStreamSequenceNumbers()
.getPartitionSequenceNumberMap()
)) {
break;
}
index--;
}
if (index == 0) {
throw new ISE("No such previous checkpoint [%s] found", checkpointMetadata);
} else if (index < checkpoints.size()) {
// if the found checkpoint is not the latest one then already checkpointed by a replica
Preconditions.checkState(index == checkpoints.size() - 1, "checkpoint consistency failure");
log.info("Already checkpointed with sequences [%s]", checkpoints.lastEntry().getValue());
return;
}
final Map<PartitionIdType, SequenceOffsetType> newCheckpoint = checkpointTaskGroup(taskGroup, false).get();
taskGroup.addNewCheckpoint(newCheckpoint);
log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId);
}
}
boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup)
{
if (taskGroup == null) {
// taskGroup might be in pendingCompletionTaskGroups or partitionGroups
if (pendingCompletionTaskGroups.containsKey(taskGroupId)) {
log.warn(
"Ignoring checkpoint request because taskGroup[%d] has already stopped indexing and is waiting for "
+ "publishing segments",
taskGroupId
);
return false;
} else if (partitionGroups.containsKey(taskGroupId)) {
log.warn("Ignoring checkpoint request because taskGroup[%d] is inactive", taskGroupId);
return false;
} else {
throw new ISE("WTH?! cannot find taskGroup [%s] among all activelyReadingTaskGroups [%s]", taskGroupId,
activelyReadingTaskGroups
);
}
}
return true;
}
}
// Map<{group RandomIdUtils}, {actively reading task group}>; see documentation for TaskGroup class
private final ConcurrentHashMap<Integer, TaskGroup> activelyReadingTaskGroups = new ConcurrentHashMap<>();
// After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [activelyReadingTaskGroups] to here so
// we can monitor its status while we queue new tasks to read the next range of sequences. This is a list since we could
// have multiple sets of tasks publishing at once if time-to-publish > taskDuration.
// Map<{group RandomIdUtils}, List<{pending completion task groups}>>
private final ConcurrentHashMap<Integer, CopyOnWriteArrayList<TaskGroup>> pendingCompletionTaskGroups = new ConcurrentHashMap<>();
// The starting sequence for a new partition in [partitionGroups] is initially set to getNotSetMarker(). When a new task group
// is created and is assigned partitions, if the sequence in [partitionGroups] is getNotSetMarker() it will take the starting
// sequence value from the metadata store, and if it can't find it there, from stream. Once a task begins
// publishing, the sequence in partitionGroups will be updated to the ending sequence of the publishing-but-not-yet-
// completed task, which will cause the next set of tasks to begin reading from where the previous task left
// off. If that previous task now fails, we will set the sequence in [partitionGroups] back to getNotSetMarker() which will
// cause successive tasks to again grab their starting sequence from metadata store. This mechanism allows us to
// start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task
// failures during publishing.
// Map<{group RandomIdUtils}, Map<{partition RandomIdUtils}, {startingOffset}>>
protected final ConcurrentHashMap<Integer, ConcurrentHashMap<PartitionIdType, SequenceOffsetType>> partitionGroups = new ConcurrentHashMap<>();
protected final ObjectMapper sortingMapper;
protected final List<PartitionIdType> partitionIds = new CopyOnWriteArrayList<>();
protected final SeekableStreamSupervisorStateManager stateManager;
protected volatile DateTime sequenceLastUpdated;
private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
protected final String dataSource;
private final Set<PartitionIdType> subsequentlyDiscoveredPartitions = new HashSet<>();
private final TaskStorage taskStorage;
private final TaskMaster taskMaster;
private final SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType> taskClient;
private final SeekableStreamSupervisorSpec spec;
private final SeekableStreamSupervisorIOConfig ioConfig;
private final SeekableStreamSupervisorTuningConfig tuningConfig;
private final SeekableStreamIndexTaskTuningConfig taskTuningConfig;
private final String supervisorId;
private final TaskInfoProvider taskInfoProvider;
private final long futureTimeoutInSeconds; // how long to wait for async operations to complete
private final RowIngestionMetersFactory rowIngestionMetersFactory;
private final ExecutorService exec;
private final ScheduledExecutorService scheduledExec;
private final ScheduledExecutorService reportingExec;
private final ListeningExecutorService workerExec;
private final BlockingQueue<Notice> notices = new LinkedBlockingDeque<>();
private final Object stopLock = new Object();
private final Object stateChangeLock = new Object();
private final Object recordSupplierLock = new Object();
private final boolean useExclusiveStartingSequence;
private boolean listenerRegistered = false;
private long lastRunTime;
private int initRetryCounter = 0;
private volatile DateTime firstRunTime;
private volatile DateTime earlyStopTime = null;
private volatile RecordSupplier<PartitionIdType, SequenceOffsetType> recordSupplier;
private volatile boolean started = false;
private volatile boolean stopped = false;
private volatile boolean lifecycleStarted = false;
public SeekableStreamSupervisor(
final String supervisorId,
final TaskStorage taskStorage,
final TaskMaster taskMaster,
final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
final SeekableStreamIndexTaskClientFactory<? extends SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType>> taskClientFactory,
final ObjectMapper mapper,
final SeekableStreamSupervisorSpec spec,
final RowIngestionMetersFactory rowIngestionMetersFactory,
final boolean useExclusiveStartingSequence
)
{
this.taskStorage = taskStorage;
this.taskMaster = taskMaster;
this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true);
this.spec = spec;
this.rowIngestionMetersFactory = rowIngestionMetersFactory;
this.useExclusiveStartingSequence = useExclusiveStartingSequence;
this.dataSource = spec.getDataSchema().getDataSource();
this.ioConfig = spec.getIoConfig();
this.tuningConfig = spec.getTuningConfig();
this.taskTuningConfig = this.tuningConfig.convertToTaskTuningConfig();
this.supervisorId = supervisorId;
this.exec = Execs.singleThreaded(supervisorId);
this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d");
this.reportingExec = Execs.scheduledSingleThreaded(supervisorId + "-Reporting-%d");
this.stateManager = new SeekableStreamSupervisorStateManager(spec.getSupervisorStateManagerConfig(), spec.isSuspended());
int workerThreads = (this.tuningConfig.getWorkerThreads() != null
? this.tuningConfig.getWorkerThreads()
: Math.min(10, this.ioConfig.getTaskCount()));
this.workerExec = MoreExecutors.listeningDecorator(Execs.multiThreaded(workerThreads, supervisorId + "-Worker-%d"));
log.info("Created worker pool with [%d] threads for dataSource [%s]", workerThreads, this.dataSource);
this.taskInfoProvider = new TaskInfoProvider()
{
@Override
public TaskLocation getTaskLocation(final String id)
{
Preconditions.checkNotNull(id, "id");
Optional<TaskRunner> taskRunner = taskMaster.getTaskRunner();
if (taskRunner.isPresent()) {
Optional<? extends TaskRunnerWorkItem> item = Iterables.tryFind(
taskRunner.get().getRunningTasks(),
(Predicate<TaskRunnerWorkItem>) taskRunnerWorkItem -> id.equals(taskRunnerWorkItem.getTaskId())
);
if (item.isPresent()) {
return item.get().getLocation();
}
} else {
log.error("Failed to get task runner because I'm not the leader!");
}
return TaskLocation.unknown();
}
@Override
public Optional<TaskStatus> getTaskStatus(String id)
{
return taskStorage.getStatus(id);
}
};
this.futureTimeoutInSeconds = Math.max(
MINIMUM_FUTURE_TIMEOUT_IN_SECONDS,
tuningConfig.getChatRetries() * (tuningConfig.getHttpTimeout().getStandardSeconds()
+ IndexTaskClient.MAX_RETRY_WAIT_SECONDS)
);
int chatThreads = (this.tuningConfig.getChatThreads() != null
? this.tuningConfig.getChatThreads()
: Math.min(10, this.ioConfig.getTaskCount() * this.ioConfig.getReplicas()));
this.taskClient = taskClientFactory.build(
taskInfoProvider,
dataSource,
chatThreads,
this.tuningConfig.getHttpTimeout(),
this.tuningConfig.getChatRetries()
);
log.info(
"Created taskClient with dataSource[%s] chatThreads[%d] httpTimeout[%s] chatRetries[%d]",
dataSource,
chatThreads,
this.tuningConfig.getHttpTimeout(),
this.tuningConfig.getChatRetries()
);
}
@Override
public void start()
{
synchronized (stateChangeLock) {
Preconditions.checkState(!lifecycleStarted, "already started");
Preconditions.checkState(!exec.isShutdown(), "already stopped");
// Try normal initialization first, if that fails then schedule periodic initialization retries
try {
tryInit();
}
catch (Exception e) {
if (!started) {
log.warn(
"First initialization attempt failed for SeekableStreamSupervisor[%s], starting retries...",
dataSource
);
exec.submit(
() -> {
try {
RetryUtils.retry(
() -> {
tryInit();
return 0;
},
(throwable) -> !started,
0,
MAX_INITIALIZATION_RETRIES,
null,
null
);
}
catch (Exception e2) {
log.makeAlert(
"Failed to initialize after %s retries, aborting. Please resubmit the supervisor spec to restart this supervisor [%s]",
MAX_INITIALIZATION_RETRIES,
supervisorId
).emit();
throw new RuntimeException(e2);
}
}
);
}
}
lifecycleStarted = true;
}
}
@Override
public void stop(boolean stopGracefully)
{
synchronized (stateChangeLock) {
Preconditions.checkState(lifecycleStarted, "lifecycle not started");
log.info("Beginning shutdown of [%s]", supervisorId);
stateManager.maybeSetState(SupervisorStateManager.BasicState.STOPPING);
try {
scheduledExec.shutdownNow(); // stop recurring executions
reportingExec.shutdownNow();
if (started) {
Optional<TaskRunner> taskRunner = taskMaster.getTaskRunner();
if (taskRunner.isPresent()) {
taskRunner.get().unregisterListener(supervisorId);
}
// Stopping gracefully will synchronize the end sequences of the tasks and signal them to publish, and will block
// until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through
// the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the
// tasks as they are.
synchronized (stopLock) {
if (stopGracefully) {
log.info("Posting GracefulShutdownNotice, signalling managed tasks to complete and publish");
notices.add(new GracefulShutdownNotice());
} else {
log.info("Posting ShutdownNotice");
notices.add(new ShutdownNotice());
}
long shutdownTimeoutMillis = tuningConfig.getShutdownTimeout().getMillis();
long endTime = System.currentTimeMillis() + shutdownTimeoutMillis;
while (!stopped) {
long sleepTime = endTime - System.currentTimeMillis();
if (sleepTime <= 0) {
log.info("Timed out while waiting for shutdown (timeout [%,dms])", shutdownTimeoutMillis);
stopped = true;
break;
}
stopLock.wait(sleepTime);
}
}
log.info("Shutdown notice handled");
}
taskClient.close();
workerExec.shutdownNow();
exec.shutdownNow();
started = false;
log.info("[%s] has stopped", supervisorId);
}
catch (Exception e) {
stateManager.recordThrowableEvent(e);
log.makeAlert(e, "Exception stopping [%s]", supervisorId)
.emit();
}
}
}
@Override
public void reset(DataSourceMetadata dataSourceMetadata)
{
log.info("Posting ResetNotice");
notices.add(new ResetNotice(dataSourceMetadata));
}
@VisibleForTesting
public void tryInit()
{
synchronized (stateChangeLock) {
if (started) {
log.warn("Supervisor was already started, skipping init");
return;
}
if (stopped) {
log.warn("Supervisor was already stopped, skipping init.");
return;
}
try {
recordSupplier = setupRecordSupplier();
exec.submit(
() -> {
try {
long pollTimeout = Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS);
while (!Thread.currentThread().isInterrupted() && !stopped) {
final Notice notice = notices.poll(pollTimeout, TimeUnit.MILLISECONDS);
if (notice == null) {
continue;
}
try {
notice.handle();
}
catch (Throwable e) {
stateManager.recordThrowableEvent(e);
log.makeAlert(e, "SeekableStreamSupervisor[%s] failed to handle notice", dataSource)
.addData("noticeClass", notice.getClass().getSimpleName())
.emit();
}
}
}
catch (InterruptedException e) {
stateManager.recordThrowableEvent(e);
log.info("SeekableStreamSupervisor[%s] interrupted, exiting", dataSource);
}
}
);
firstRunTime = DateTimes.nowUtc().plus(ioConfig.getStartDelay());
scheduledExec.scheduleAtFixedRate(
buildRunTask(),
ioConfig.getStartDelay().getMillis(),
Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS),
TimeUnit.MILLISECONDS
);
scheduleReporting(reportingExec);
started = true;
log.info(
"Started SeekableStreamSupervisor[%s], first run in [%s], with spec: [%s]",
dataSource,
ioConfig.getStartDelay(),
spec.toString()
);
}
catch (Exception e) {
stateManager.recordThrowableEvent(e);
if (recordSupplier != null) {
recordSupplier.close();
}
initRetryCounter++;
log.makeAlert(e, "Exception starting SeekableStreamSupervisor[%s]", dataSource)
.emit();
throw new RuntimeException(e);
}
}
}
private Runnable buildRunTask()
{
return () -> notices.add(new RunNotice());
}
@Override
public SupervisorReport getStatus()
{
return generateReport(true);
}
@Override
public SupervisorStateManager.State getState()
{
return stateManager.getSupervisorState();
}
@Override
public Boolean isHealthy()
{
return stateManager.isHealthy();
}
private SupervisorReport<? extends SeekableStreamSupervisorReportPayload<PartitionIdType, SequenceOffsetType>> generateReport(
boolean includeOffsets
)
{
int numPartitions = partitionGroups.values().stream().mapToInt(Map::size).sum();
final SeekableStreamSupervisorReportPayload<PartitionIdType, SequenceOffsetType> payload = createReportPayload(
numPartitions,
includeOffsets
);
SupervisorReport<SeekableStreamSupervisorReportPayload<PartitionIdType, SequenceOffsetType>> report = new SupervisorReport<>(
dataSource,
DateTimes.nowUtc(),
payload
);
List<TaskReportData<PartitionIdType, SequenceOffsetType>> taskReports = new ArrayList<>();
try {
for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) {
for (Entry<String, TaskData> entry : taskGroup.tasks.entrySet()) {
String taskId = entry.getKey();
@Nullable
DateTime startTime = entry.getValue().startTime;
Map<PartitionIdType, SequenceOffsetType> currentOffsets = entry.getValue().currentSequences;
Long remainingSeconds = null;
if (startTime != null) {
long elapsedMillis = System.currentTimeMillis() - startTime.getMillis();
long remainingMillis = Math.max(0, ioConfig.getTaskDuration().getMillis() - elapsedMillis);
remainingSeconds = TimeUnit.MILLISECONDS.toSeconds(remainingMillis);
}
taskReports.add(
new TaskReportData<>(
taskId,
includeOffsets ? taskGroup.startingSequences : null,
includeOffsets ? currentOffsets : null,
startTime,
remainingSeconds,
TaskReportData.TaskType.ACTIVE,
includeOffsets ? getLagPerPartition(currentOffsets) : null
)
);
}
}
for (List<TaskGroup> taskGroups : pendingCompletionTaskGroups.values()) {
for (TaskGroup taskGroup : taskGroups) {
for (Entry<String, TaskData> entry : taskGroup.tasks.entrySet()) {
String taskId = entry.getKey();
@Nullable
DateTime startTime = entry.getValue().startTime;
Map<PartitionIdType, SequenceOffsetType> currentOffsets = entry.getValue().currentSequences;
Long remainingSeconds = null;
if (taskGroup.completionTimeout != null) {
remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis())
/ 1000;
}
taskReports.add(
new TaskReportData<>(
taskId,
includeOffsets ? taskGroup.startingSequences : null,
includeOffsets ? currentOffsets : null,
startTime,
remainingSeconds,
TaskReportData.TaskType.PUBLISHING,
null
)
);
}
}
}
taskReports.forEach(payload::addTask);
}
catch (Exception e) {
log.warn(e, "Failed to generate status report");
}
return report;
}
@Override
public Map<String, Map<String, Object>> getStats()
{
try {
return getCurrentTotalStats();
}
catch (InterruptedException ie) {
Thread.currentThread().interrupt();
log.error(ie, "getStats() interrupted.");
throw new RuntimeException(ie);
}
catch (ExecutionException | TimeoutException eete) {
throw new RuntimeException(eete);
}
}
/**
* Collect row ingestion stats from all tasks managed by this supervisor.
*
* @return A map of groupId->taskId->task row stats
*
* @throws InterruptedException
* @throws ExecutionException
* @throws TimeoutException
*/
private Map<String, Map<String, Object>> getCurrentTotalStats()
throws InterruptedException, ExecutionException, TimeoutException
{
Map<String, Map<String, Object>> allStats = new HashMap<>();
final List<ListenableFuture<StatsFromTaskResult>> futures = new ArrayList<>();
final List<Pair<Integer, String>> groupAndTaskIds = new ArrayList<>();
for (int groupId : activelyReadingTaskGroups.keySet()) {
TaskGroup group = activelyReadingTaskGroups.get(groupId);
for (String taskId : group.taskIds()) {
futures.add(
Futures.transform(
taskClient.getMovingAveragesAsync(taskId),
(Function<Map<String, Object>, StatsFromTaskResult>) (currentStats) -> new StatsFromTaskResult(
groupId,
taskId,
currentStats
)
)
);
groupAndTaskIds.add(new Pair<>(groupId, taskId));
}
}
for (int groupId : pendingCompletionTaskGroups.keySet()) {
List<TaskGroup> pendingGroups = pendingCompletionTaskGroups.get(groupId);
for (TaskGroup pendingGroup : pendingGroups) {
for (String taskId : pendingGroup.taskIds()) {
futures.add(
Futures.transform(
taskClient.getMovingAveragesAsync(taskId),
(Function<Map<String, Object>, StatsFromTaskResult>) (currentStats) -> new StatsFromTaskResult(
groupId,
taskId,
currentStats
)
)
);
groupAndTaskIds.add(new Pair<>(groupId, taskId));
}
}
}
List<StatsFromTaskResult> results = Futures.successfulAsList(futures)
.get(futureTimeoutInSeconds, TimeUnit.SECONDS);
for (int i = 0; i < results.size(); i++) {
StatsFromTaskResult result = results.get(i);
if (result != null) {
Map<String, Object> groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> new HashMap<>());
groupMap.put(result.getTaskId(), result.getStats());
} else {
Pair<Integer, String> groupAndTaskId = groupAndTaskIds.get(i);
log.error("Failed to get stats for group[%d]-task[%s]", groupAndTaskId.lhs, groupAndTaskId.rhs);
}
}
return allStats;
}
@VisibleForTesting
public void addTaskGroupToActivelyReadingTaskGroup(
int taskGroupId,
ImmutableMap<PartitionIdType, SequenceOffsetType> partitionOffsets,
Optional<DateTime> minMsgTime,
Optional<DateTime> maxMsgTime,
Set<String> tasks,
Set<PartitionIdType> exclusiveStartingSequencePartitions
)
{
TaskGroup group = new TaskGroup(
taskGroupId,
partitionOffsets,
null,
minMsgTime,
maxMsgTime,
exclusiveStartingSequencePartitions
);
group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData())));
if (activelyReadingTaskGroups.putIfAbsent(taskGroupId, group) != null) {
throw new ISE(
"trying to add taskGroup with RandomIdUtils [%s] to actively reading task groups, but group already exists.",
taskGroupId
);
}
}
@VisibleForTesting
public void addTaskGroupToPendingCompletionTaskGroup(
int taskGroupId,
ImmutableMap<PartitionIdType, SequenceOffsetType> partitionOffsets,
Optional<DateTime> minMsgTime,
Optional<DateTime> maxMsgTime,
Set<String> tasks,
Set<PartitionIdType> exclusiveStartingSequencePartitions