-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
SupervisorManager.java
439 lines (381 loc) · 15.3 KB
/
SupervisorManager.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
/*
* 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.supervisor;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler;
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec;
import org.apache.druid.java.util.common.Pair;
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.metadata.MetadataSupervisorManager;
import org.apache.druid.metadata.PendingSegmentRecord;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.segment.incremental.ParseExceptionReport;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
/**
* Manages the creation and lifetime of {@link Supervisor}.
*/
public class SupervisorManager
{
private static final EmittingLogger log = new EmittingLogger(SupervisorManager.class);
private final MetadataSupervisorManager metadataSupervisorManager;
private final ConcurrentHashMap<String, Pair<Supervisor, SupervisorSpec>> supervisors = new ConcurrentHashMap<>();
// SupervisorTaskAutoScaler could be null
private final ConcurrentHashMap<String, SupervisorTaskAutoScaler> autoscalers = new ConcurrentHashMap<>();
private final Object lock = new Object();
private volatile boolean started = false;
@Inject
public SupervisorManager(MetadataSupervisorManager metadataSupervisorManager)
{
this.metadataSupervisorManager = metadataSupervisorManager;
}
public MetadataSupervisorManager getMetadataSupervisorManager()
{
return metadataSupervisorManager;
}
public Set<String> getSupervisorIds()
{
return supervisors.keySet();
}
/**
* @param datasource Datasource to find active supervisor id with append lock for.
* @return An optional with the active appending supervisor id if it exists.
*/
public Optional<String> getActiveSupervisorIdForDatasourceWithAppendLock(String datasource)
{
for (Map.Entry<String, Pair<Supervisor, SupervisorSpec>> entry : supervisors.entrySet()) {
final String supervisorId = entry.getKey();
final Supervisor supervisor = entry.getValue().lhs;
final SupervisorSpec supervisorSpec = entry.getValue().rhs;
boolean hasAppendLock = Tasks.DEFAULT_USE_CONCURRENT_LOCKS;
if (supervisorSpec instanceof SeekableStreamSupervisorSpec) {
SeekableStreamSupervisorSpec seekableStreamSupervisorSpec = (SeekableStreamSupervisorSpec) supervisorSpec;
Map<String, Object> context = seekableStreamSupervisorSpec.getContext();
if (context != null) {
Boolean useConcurrentLocks = QueryContexts.getAsBoolean(
Tasks.USE_CONCURRENT_LOCKS,
context.get(Tasks.USE_CONCURRENT_LOCKS)
);
if (useConcurrentLocks == null) {
TaskLockType taskLockType = QueryContexts.getAsEnum(
Tasks.TASK_LOCK_TYPE,
context.get(Tasks.TASK_LOCK_TYPE),
TaskLockType.class
);
if (taskLockType == null) {
hasAppendLock = Tasks.DEFAULT_USE_CONCURRENT_LOCKS;
} else if (taskLockType == TaskLockType.APPEND) {
hasAppendLock = true;
} else {
hasAppendLock = false;
}
} else {
hasAppendLock = useConcurrentLocks;
}
}
}
if (supervisor instanceof SeekableStreamSupervisor
&& !supervisorSpec.isSuspended()
&& supervisorSpec.getDataSources().contains(datasource)
&& (hasAppendLock)) {
return Optional.of(supervisorId);
}
}
return Optional.absent();
}
public Optional<SupervisorSpec> getSupervisorSpec(String id)
{
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.rhs);
}
public Optional<SupervisorStateManager.State> getSupervisorState(String id)
{
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.lhs.getState());
}
public boolean createOrUpdateAndStartSupervisor(SupervisorSpec spec)
{
Preconditions.checkState(started, "SupervisorManager not started");
Preconditions.checkNotNull(spec, "spec");
Preconditions.checkNotNull(spec.getId(), "spec.getId()");
Preconditions.checkNotNull(spec.getDataSources(), "spec.getDatasources()");
synchronized (lock) {
Preconditions.checkState(started, "SupervisorManager not started");
possiblyStopAndRemoveSupervisorInternal(spec.getId(), false);
return createAndStartSupervisorInternal(spec, true);
}
}
public boolean stopAndRemoveSupervisor(String id)
{
Preconditions.checkState(started, "SupervisorManager not started");
Preconditions.checkNotNull(id, "id");
synchronized (lock) {
Preconditions.checkState(started, "SupervisorManager not started");
return possiblyStopAndRemoveSupervisorInternal(id, true);
}
}
public boolean suspendOrResumeSupervisor(String id, boolean suspend)
{
Preconditions.checkState(started, "SupervisorManager not started");
Preconditions.checkNotNull(id, "id");
synchronized (lock) {
Preconditions.checkState(started, "SupervisorManager not started");
return possiblySuspendOrResumeSupervisorInternal(id, suspend);
}
}
@LifecycleStart
public void start()
{
Preconditions.checkState(!started, "SupervisorManager already started");
log.info("Loading stored supervisors from database");
synchronized (lock) {
Map<String, SupervisorSpec> supervisors = metadataSupervisorManager.getLatest();
for (Map.Entry<String, SupervisorSpec> supervisor : supervisors.entrySet()) {
final SupervisorSpec spec = supervisor.getValue();
if (!(spec instanceof NoopSupervisorSpec)) {
try {
createAndStartSupervisorInternal(spec, false);
}
catch (Exception ex) {
log.error(ex, "Failed to start supervisor: id [%s]", spec.getId());
}
}
}
started = true;
}
}
@LifecycleStop
public void stop()
{
Preconditions.checkState(started, "SupervisorManager not started");
synchronized (lock) {
for (String id : supervisors.keySet()) {
try {
supervisors.get(id).lhs.stop(false);
SupervisorTaskAutoScaler autoscaler = autoscalers.get(id);
if (autoscaler != null) {
autoscaler.stop();
}
}
catch (Exception e) {
log.warn(e, "Caught exception while stopping supervisor [%s]", id);
}
}
supervisors.clear();
autoscalers.clear();
started = false;
}
log.info("SupervisorManager stopped.");
}
public List<VersionedSupervisorSpec> getSupervisorHistoryForId(String id)
{
return metadataSupervisorManager.getAllForId(id);
}
public Map<String, List<VersionedSupervisorSpec>> getSupervisorHistory()
{
return metadataSupervisorManager.getAll();
}
public Optional<SupervisorReport> getSupervisorStatus(String id)
{
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.lhs.getStatus());
}
public Optional<Map<String, Map<String, Object>>> getSupervisorStats(String id)
{
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.lhs.getStats());
}
public Optional<List<ParseExceptionReport>> getSupervisorParseErrors(String id)
{
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.lhs.getParseErrors());
}
public Optional<Boolean> isSupervisorHealthy(String id)
{
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.lhs.isHealthy());
}
public boolean resetSupervisor(String id, @Nullable DataSourceMetadata resetDataSourceMetadata)
{
Preconditions.checkState(started, "SupervisorManager not started");
Preconditions.checkNotNull(id, "id");
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
if (supervisor == null) {
return false;
}
if (resetDataSourceMetadata == null) {
supervisor.lhs.reset(null);
} else {
supervisor.lhs.resetOffsets(resetDataSourceMetadata);
}
SupervisorTaskAutoScaler autoscaler = autoscalers.get(id);
if (autoscaler != null) {
autoscaler.reset();
}
return true;
}
public boolean checkPointDataSourceMetadata(
String supervisorId,
int taskGroupId,
DataSourceMetadata previousDataSourceMetadata
)
{
try {
Preconditions.checkState(started, "SupervisorManager not started");
Preconditions.checkNotNull(supervisorId, "supervisorId cannot be null");
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(supervisorId);
Preconditions.checkNotNull(supervisor, "supervisor could not be found");
supervisor.lhs.checkpoint(taskGroupId, previousDataSourceMetadata);
return true;
}
catch (Exception e) {
log.error(e, "Checkpoint request failed");
}
return false;
}
/**
* Registers a new version of the given pending segment on a supervisor. This
* allows the supervisor to include the pending segment in queries fired against
* that segment version.
*/
public boolean registerUpgradedPendingSegmentOnSupervisor(
String supervisorId,
PendingSegmentRecord upgradedPendingSegment
)
{
try {
Preconditions.checkNotNull(supervisorId, "supervisorId cannot be null");
Preconditions.checkNotNull(upgradedPendingSegment, "upgraded pending segment cannot be null");
Preconditions.checkNotNull(upgradedPendingSegment.getTaskAllocatorId(), "taskAllocatorId cannot be null");
Preconditions.checkNotNull(
upgradedPendingSegment.getUpgradedFromSegmentId(),
"upgradedFromSegmentId cannot be null"
);
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(supervisorId);
Preconditions.checkNotNull(supervisor, "supervisor could not be found");
if (!(supervisor.lhs instanceof SeekableStreamSupervisor)) {
return false;
}
SeekableStreamSupervisor<?, ?, ?> seekableStreamSupervisor = (SeekableStreamSupervisor<?, ?, ?>) supervisor.lhs;
seekableStreamSupervisor.registerNewVersionOfPendingSegment(upgradedPendingSegment);
return true;
}
catch (Exception e) {
log.error(e, "Failed to upgrade pending segment[%s] to new pending segment[%s] on Supervisor[%s].",
upgradedPendingSegment.getUpgradedFromSegmentId(), upgradedPendingSegment.getId().getVersion(), supervisorId);
}
return false;
}
/**
* Stops a supervisor with a given id and then removes it from the list.
* <p/>
* Caller should have acquired [lock] before invoking this method to avoid contention with other threads that may be
* starting, stopping, suspending and resuming supervisors.
*
* @return true if a supervisor was stopped, false if there was no supervisor with this id
*/
private boolean possiblyStopAndRemoveSupervisorInternal(String id, boolean writeTombstone)
{
Pair<Supervisor, SupervisorSpec> pair = supervisors.get(id);
if (pair == null) {
return false;
}
if (writeTombstone) {
metadataSupervisorManager.insert(
id,
new NoopSupervisorSpec(null, pair.rhs.getDataSources())
); // where NoopSupervisorSpec is a tombstone
}
pair.lhs.stop(true);
supervisors.remove(id);
SupervisorTaskAutoScaler autoscler = autoscalers.get(id);
if (autoscler != null) {
autoscler.stop();
autoscalers.remove(id);
}
return true;
}
/**
* Suspend or resume a supervisor with a given id.
* <p/>
* Caller should have acquired [lock] before invoking this method to avoid contention with other threads that may be
* starting, stopping, suspending and resuming supervisors.
*
* @return true if a supervisor was suspended or resumed, false if there was no supervisor with this id
* or suspend a suspended supervisor or resume a running supervisor
*/
private boolean possiblySuspendOrResumeSupervisorInternal(String id, boolean suspend)
{
Pair<Supervisor, SupervisorSpec> pair = supervisors.get(id);
if (pair == null || pair.rhs.isSuspended() == suspend) {
return false;
}
SupervisorSpec nextState = suspend ? pair.rhs.createSuspendedSpec() : pair.rhs.createRunningSpec();
possiblyStopAndRemoveSupervisorInternal(nextState.getId(), false);
return createAndStartSupervisorInternal(nextState, true);
}
/**
* Creates a supervisor from the provided spec and starts it if there is not already a supervisor with that id.
* <p/>
* Caller should have acquired [lock] before invoking this method to avoid contention with other threads that may be
* starting, stopping, suspending and resuming supervisors.
*
* @return true if a new supervisor was created, false if there was already an existing supervisor with this id
*/
private boolean createAndStartSupervisorInternal(SupervisorSpec spec, boolean persistSpec)
{
String id = spec.getId();
if (supervisors.containsKey(id)) {
return false;
}
Supervisor supervisor;
SupervisorTaskAutoScaler autoscaler;
try {
supervisor = spec.createSupervisor();
autoscaler = spec.createAutoscaler(supervisor);
supervisor.start();
if (autoscaler != null) {
autoscaler.start();
}
}
catch (Exception e) {
log.error("Failed to create and start supervisor: [%s]", spec.getId());
throw new RuntimeException(e);
}
if (persistSpec) {
metadataSupervisorManager.insert(id, spec);
}
supervisors.put(id, Pair.of(supervisor, spec));
if (autoscaler != null) {
autoscalers.put(id, autoscaler);
}
return true;
}
}