-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
SeekableStreamIndexTaskClientAsyncImpl.java
691 lines (629 loc) · 24.4 KB
/
SeekableStreamIndexTaskClientAsyncImpl.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
/*
* 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;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.type.MapType;
import com.fasterxml.jackson.databind.type.TypeFactory;
import com.google.common.base.Optional;
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.MoreExecutors;
import com.google.common.util.concurrent.SettableFuture;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.RetryPolicy;
import org.apache.druid.indexing.common.RetryPolicyConfig;
import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.TaskInfoProvider;
import org.apache.druid.java.util.common.Either;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler;
import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
import org.apache.druid.metadata.PendingSegmentRecord;
import org.apache.druid.rpc.HttpResponseException;
import org.apache.druid.rpc.IgnoreHttpResponseHandler;
import org.apache.druid.rpc.RequestBuilder;
import org.apache.druid.rpc.ServiceClient;
import org.apache.druid.rpc.ServiceClientFactory;
import org.apache.druid.rpc.ServiceClosedException;
import org.apache.druid.rpc.ServiceLocation;
import org.apache.druid.rpc.ServiceLocations;
import org.apache.druid.rpc.ServiceLocator;
import org.apache.druid.rpc.ServiceNotAvailableException;
import org.apache.druid.rpc.ServiceRetryPolicy;
import org.apache.druid.rpc.StandardRetryPolicy;
import org.apache.druid.rpc.indexing.SpecificTaskRetryPolicy;
import org.apache.druid.segment.incremental.ParseExceptionReport;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
/**
* Implementation of {@link SeekableStreamIndexTaskClient} based on {@link ServiceClient}.
*/
public abstract class SeekableStreamIndexTaskClientAsyncImpl<PartitionIdType, SequenceOffsetType>
implements SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType>
{
private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClientAsyncImpl.class);
public static final int MIN_RETRY_WAIT_SECONDS = 2;
public static final int MAX_RETRY_WAIT_SECONDS = 10;
private final ServiceClientFactory serviceClientFactory;
private final TaskInfoProvider taskInfoProvider;
private final ObjectMapper jsonMapper;
private final Duration httpTimeout;
private final long httpRetries;
// Used by getOffsetsWhenPaused, due to special retry logic.
private final ScheduledExecutorService retryExec;
public SeekableStreamIndexTaskClientAsyncImpl(
final String dataSource,
final ServiceClientFactory serviceClientFactory,
final TaskInfoProvider taskInfoProvider,
final ObjectMapper jsonMapper,
final Duration httpTimeout,
final long httpRetries
)
{
this.serviceClientFactory = serviceClientFactory;
this.taskInfoProvider = taskInfoProvider;
this.jsonMapper = jsonMapper;
this.httpTimeout = httpTimeout;
this.httpRetries = httpRetries;
this.retryExec = Execs.scheduledSingleThreaded(
StringUtils.format(
"%s-%s-%%d",
getClass().getSimpleName(),
StringUtils.encodeForFormat(dataSource)
)
);
}
@Override
@SuppressWarnings("unchecked")
public ListenableFuture<TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>>> getCheckpointsAsync(
final String id,
final boolean retry
)
{
return makeRequest(id, new RequestBuilder(HttpMethod.GET, "/checkpoints"))
.handler(new BytesFullResponseHandler())
.onSuccess(r -> {
final TypeFactory factory = jsonMapper.getTypeFactory();
return (TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>>)
JacksonUtils.readValue(
jsonMapper,
r.getContent(),
factory.constructMapType(
TreeMap.class,
factory.constructType(Integer.class),
factory.constructMapType(Map.class, getPartitionType(), getSequenceType())
)
);
})
.onNotAvailable(e -> Either.value(new TreeMap<>()))
.retry(retry)
.go();
}
@Override
public ListenableFuture<Boolean> stopAsync(final String id, final boolean publish)
{
return makeRequest(id, new RequestBuilder(HttpMethod.POST, "/stop" + (publish ? "?publish=true" : "")))
.onSuccess(r -> true)
.onHttpError(e -> {
log.warn("Task [%s] coundln't be stopped because of http request failure [%s].", id, e.getMessage());
return Either.value(false);
})
.onNotAvailable(e -> {
log.warn("Task [%s] coundln't be stopped because it is not available.", id);
return Either.value(false);
})
.onClosed(e -> {
log.warn("Task [%s] couldn't be stopped because it is no longer running.", id);
return Either.value(true);
})
.go();
}
@Override
public ListenableFuture<Boolean> resumeAsync(final String id)
{
return makeRequest(id, new RequestBuilder(HttpMethod.POST, "/resume"))
.onSuccess(r -> true)
.onException(e -> Either.value(false))
.go();
}
@Override
public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getCurrentOffsetsAsync(String id, boolean retry)
{
return makeRequest(id, new RequestBuilder(HttpMethod.GET, "/offsets/current"))
.handler(new BytesFullResponseHandler())
.onSuccess(r -> deserializeOffsetsMap(r.getContent()))
.onNotAvailable(e -> Either.value(Collections.emptyMap()))
.retry(retry)
.go();
}
@Override
public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getEndOffsetsAsync(String id)
{
return makeRequest(id, new RequestBuilder(HttpMethod.GET, "/offsets/end"))
.handler(new BytesFullResponseHandler())
.onSuccess(r -> deserializeOffsetsMap(r.getContent()))
.onNotAvailable(e -> Either.value(Collections.emptyMap()))
.go();
}
@Override
public ListenableFuture<Boolean> registerNewVersionOfPendingSegmentAsync(
String taskId,
PendingSegmentRecord pendingSegmentRecord
)
{
final RequestBuilder requestBuilder
= new RequestBuilder(HttpMethod.POST, "/pendingSegmentVersion")
.jsonContent(jsonMapper, pendingSegmentRecord);
return makeRequest(taskId, requestBuilder)
.handler(IgnoreHttpResponseHandler.INSTANCE)
.onSuccess(r -> true)
.go();
}
@Override
public ListenableFuture<Boolean> setEndOffsetsAsync(
final String id,
final Map<PartitionIdType, SequenceOffsetType> endOffsets,
final boolean finalize
)
{
final RequestBuilder requestBuilder = new RequestBuilder(
HttpMethod.POST,
StringUtils.format("/offsets/end?finish=%s", finalize)
).jsonContent(jsonMapper, endOffsets);
return makeRequest(id, requestBuilder)
.handler(IgnoreHttpResponseHandler.INSTANCE)
.onSuccess(r -> true)
.go();
}
@Override
public ListenableFuture<SeekableStreamIndexTaskRunner.Status> getStatusAsync(final String id)
{
return makeRequest(id, new RequestBuilder(HttpMethod.GET, "/status"))
.handler(new BytesFullResponseHandler())
.onSuccess(
r ->
JacksonUtils.readValue(jsonMapper, r.getContent(), SeekableStreamIndexTaskRunner.Status.class)
)
.onNotAvailable(e -> Either.value(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.go();
}
@Override
public ListenableFuture<DateTime> getStartTimeAsync(String id)
{
return makeRequest(id, new RequestBuilder(HttpMethod.GET, "/time/start"))
.handler(new BytesFullResponseHandler())
.onSuccess(r -> {
if (isNullOrEmpty(r.getContent())) {
return null;
} else {
return JacksonUtils.readValue(jsonMapper, r.getContent(), DateTime.class);
}
})
.onNotAvailable(e -> Either.value(null))
.go();
}
@Override
public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> pauseAsync(String id)
{
final ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> pauseFuture =
makeRequest(id, new RequestBuilder(HttpMethod.POST, "/pause"))
.handler(new BytesFullResponseHandler())
.onSuccess(r -> {
if (r.getStatus().equals(HttpResponseStatus.OK)) {
log.info("Task [%s] paused successfully", id);
return deserializeOffsetsMap(r.getContent());
} else if (r.getStatus().equals(HttpResponseStatus.ACCEPTED)) {
// Return null, which triggers a loop later to wait for the task to enter PAUSED state.
return null;
} else {
throw new ISE(
"Pause request for task [%s] failed with response [%s]",
id,
r.getStatus()
);
}
})
.onNotAvailable(e -> Either.value(Collections.emptyMap()))
.go();
return FutureUtils.transformAsync(
pauseFuture,
result -> {
if (result != null) {
return Futures.immediateFuture(result);
} else {
return getOffsetsWhenPaused(
id,
new RetryPolicyFactory(
new RetryPolicyConfig()
.setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS))
.setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS))
.setMaxRetryCount(httpRetries)
).makeRetryPolicy()
);
}
}
);
}
@Override
public ListenableFuture<Map<String, Object>> getMovingAveragesAsync(String id)
{
return makeRequest(id, new RequestBuilder(HttpMethod.GET, "/rowStats"))
.handler(new BytesFullResponseHandler())
.onSuccess(r -> {
if (isNullOrEmpty(r.getContent())) {
log.warn("Got empty response when calling getMovingAverages, id[%s]", id);
return null;
} else {
return JacksonUtils.readValue(jsonMapper, r.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
}
})
.onNotAvailable(e -> Either.value(Collections.emptyMap()))
.go();
}
@Override
public ListenableFuture<List<ParseExceptionReport>> getParseErrorsAsync(String id)
{
return makeRequest(id, new RequestBuilder(HttpMethod.GET, "/unparseableEvents"))
.handler(new BytesFullResponseHandler())
.onSuccess(r -> {
if (isNullOrEmpty(r.getContent())) {
log.warn("Got empty response when calling getParseErrors, id[%s]", id);
return null;
} else {
return JacksonUtils.readValue(
jsonMapper,
r.getContent(),
TYPE_REFERENCE_LIST_PARSE_EXCEPTION_REPORT
);
}
})
.onNotAvailable(e -> Either.value(Collections.emptyList()))
.go();
}
@Override
public void close()
{
retryExec.shutdownNow();
}
/**
* Create a {@link SeekableStreamRequestBuilder}.
*/
private SeekableStreamRequestBuilder<Void, Void, Void> makeRequest(
String taskId,
RequestBuilder requestBuilder
)
{
return new SeekableStreamRequestBuilder<>(
taskId,
requestBuilder,
IgnoreHttpResponseHandler.INSTANCE,
Function.identity()
);
}
/**
* Helper for deserializing offset maps.
*/
private Map<PartitionIdType, SequenceOffsetType> deserializeOffsetsMap(final byte[] content)
{
final MapType offsetsMapType =
jsonMapper.getTypeFactory().constructMapType(Map.class, getPartitionType(), getSequenceType());
return JacksonUtils.readValue(jsonMapper, content, offsetsMapType);
}
/**
* Helper for {@link #pauseAsync}.
*
* Calls {@link #getStatusAsync} in a loop until a task is paused, then calls {@link #getCurrentOffsetsAsync} to
* get the post-pause offsets for the task.
*/
private ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getOffsetsWhenPaused(
final String taskId,
final RetryPolicy retryPolicy
)
{
final ListenableFuture<SeekableStreamIndexTaskRunner.Status> statusFuture = getStatusAsync(taskId);
return FutureUtils.transformAsync(
statusFuture,
status -> {
if (status == SeekableStreamIndexTaskRunner.Status.PAUSED) {
return getCurrentOffsetsAsync(taskId, true);
} else {
final Duration delay;
synchronized (retryPolicy) {
delay = retryPolicy.getAndIncrementRetryDelay();
}
if (delay == null) {
return Futures.immediateFailedFuture(
new ISE(
"Task [%s] failed to change its status from [%s] to [%s], aborting",
taskId,
status,
SeekableStreamIndexTaskRunner.Status.PAUSED
)
);
} else {
final long sleepTime = delay.getMillis();
final SettableFuture<Map<PartitionIdType, SequenceOffsetType>> retVal = SettableFuture.create();
retryExec.schedule(
() ->
Futures.addCallback(
getOffsetsWhenPaused(taskId, retryPolicy),
new FutureCallback<Map<PartitionIdType, SequenceOffsetType>>()
{
@Override
public void onSuccess(@Nullable Map<PartitionIdType, SequenceOffsetType> result)
{
retVal.set(result);
}
@Override
public void onFailure(Throwable t)
{
retVal.setException(t);
}
},
MoreExecutors.directExecutor()
),
sleepTime,
TimeUnit.MILLISECONDS
);
return retVal;
}
}
}
);
}
private static boolean isNullOrEmpty(@Nullable final byte[] content)
{
return content == null || content.length == 0;
}
/**
* Helper for setting up each request's desired response, error handling, and retry behavior.
*/
private class SeekableStreamRequestBuilder<IntermediateType, FinalType, T>
{
private final String taskId;
private final RequestBuilder requestBuilder;
private final List<Function<Throwable, Either<Throwable, T>>> exceptionMappers = new ArrayList<>();
private HttpResponseHandler<IntermediateType, FinalType> responseHandler;
private Function<FinalType, T> responseTransformer;
private boolean retry = true;
SeekableStreamRequestBuilder(
String taskId,
RequestBuilder requestBuilder,
HttpResponseHandler<IntermediateType, FinalType> responseHandler,
Function<FinalType, T> responseTransformer
)
{
this.taskId = taskId;
this.requestBuilder = requestBuilder;
this.responseHandler = responseHandler;
this.responseTransformer = responseTransformer;
}
/**
* Handler for requests. The result from this handler is fed into the transformer provided by {@link #onSuccess}.
*/
@SuppressWarnings({"unchecked", "rawtypes"})
public <NewIntermediateType, NewFinalType> SeekableStreamRequestBuilder<NewIntermediateType, NewFinalType, T> handler(
final HttpResponseHandler<NewIntermediateType, NewFinalType> handler
)
{
this.responseHandler = (HttpResponseHandler) handler;
return (SeekableStreamRequestBuilder) this;
}
/**
* Response mapping for successful requests.
*/
@SuppressWarnings({"unchecked", "rawtypes"})
public <NewT> SeekableStreamRequestBuilder<IntermediateType, FinalType, NewT> onSuccess(
final Function<FinalType, NewT> responseTransformer
)
{
this.responseTransformer = (Function) responseTransformer;
return (SeekableStreamRequestBuilder) this;
}
/**
* Whether the request should be retried on failure. Default is true.
*/
public SeekableStreamRequestBuilder<IntermediateType, FinalType, T> retry(boolean retry)
{
this.retry = retry;
return this;
}
/**
* Error mapping for all exceptions.
*/
public SeekableStreamRequestBuilder<IntermediateType, FinalType, T> onException(final Function<Throwable, Either<Throwable, T>> fn)
{
exceptionMappers.add(fn);
return this;
}
/**
* Error mapping for {@link HttpResponseException}, which occurs when a task returns a non-2xx HTTP code.
*/
public SeekableStreamRequestBuilder<IntermediateType, FinalType, T> onHttpError(final Function<HttpResponseException, Either<Throwable, T>> fn)
{
return onException(e -> {
if (e instanceof HttpResponseException) {
return fn.apply((HttpResponseException) e);
} else {
return Either.error(e);
}
});
}
/**
* Error mapping for {@link ServiceNotAvailableException}, which occurs when a task is not available.
*/
public SeekableStreamRequestBuilder<IntermediateType, FinalType, T> onNotAvailable(final Function<ServiceNotAvailableException, Either<Throwable, T>> fn)
{
return onException(e -> {
if (e instanceof ServiceNotAvailableException) {
return fn.apply((ServiceNotAvailableException) e);
} else {
return Either.error(e);
}
});
}
/**
* Error mapping for {@link ServiceClosedException}, which occurs when a task is not running.
*/
public SeekableStreamRequestBuilder<IntermediateType, FinalType, T> onClosed(final Function<ServiceClosedException, Either<Throwable, T>> fn)
{
return onException(e -> {
if (e instanceof ServiceClosedException) {
return fn.apply((ServiceClosedException) e);
} else {
return Either.error(e);
}
});
}
/**
* Issue the request.
*/
public ListenableFuture<T> go()
{
final ServiceClient client = makeClient(taskId, retry);
final SettableFuture<T> retVal = SettableFuture.create();
Futures.addCallback(
FutureUtils.transform(
client.asyncRequest(requestBuilder.timeout(httpTimeout), responseHandler),
responseTransformer
),
new FutureCallback<T>()
{
@Override
public void onSuccess(@Nullable T result)
{
retVal.set(result);
}
@Override
public void onFailure(Throwable t)
{
Either<Throwable, T> either = Either.error(t);
for (final Function<Throwable, Either<Throwable, T>> exceptionMapper : exceptionMappers) {
if (!either.isError()) {
break;
}
try {
final Either<Throwable, T> nextEither = exceptionMapper.apply(either.error());
if (nextEither != null) {
either = nextEither;
}
}
catch (Throwable e) {
// Not expected: on-error function should never throw exceptions. Continue mapping.
log.warn(e, "Failed to map exception encountered while contacting task [%s]", taskId);
}
}
if (either.isError()) {
retVal.setException(either.error());
} else {
retVal.set(either.valueOrThrow());
}
}
},
MoreExecutors.directExecutor()
);
return retVal;
}
private ServiceClient makeClient(final String taskId, final boolean retry)
{
final ServiceRetryPolicy retryPolicy = makeRetryPolicy(taskId, retry);
// We're creating a new locator for each request and not closing it. This is OK, since SeekableStreamTaskLocator
// is stateless, cheap to create, and its close() method does nothing.
final SeekableStreamTaskLocator locator = new SeekableStreamTaskLocator(taskInfoProvider, taskId);
// We're creating a new client for each request. This is OK, clients are cheap to create and do not contain
// state that is important for us to retain across requests. (The main state they retain is preferred location
// from prior redirects; but tasks don't do redirects.)
return serviceClientFactory.makeClient(taskId, locator, retryPolicy);
}
private ServiceRetryPolicy makeRetryPolicy(final String taskId, final boolean retry)
{
final StandardRetryPolicy baseRetryPolicy;
if (retry) {
baseRetryPolicy = StandardRetryPolicy.builder()
.maxAttempts(httpRetries + 1)
.minWaitMillis(MIN_RETRY_WAIT_SECONDS * 1000)
.maxWaitMillis(MAX_RETRY_WAIT_SECONDS * 1000)
.retryNotAvailable(false)
.build();
} else {
baseRetryPolicy = StandardRetryPolicy.noRetries();
}
return new SpecificTaskRetryPolicy(taskId, baseRetryPolicy);
}
}
static class SeekableStreamTaskLocator implements ServiceLocator
{
private static final String BASE_PATH = "/druid/worker/v1/chat";
private final TaskInfoProvider taskInfoProvider;
private final String taskId;
SeekableStreamTaskLocator(TaskInfoProvider taskInfoProvider, String taskId)
{
this.taskInfoProvider = taskInfoProvider;
this.taskId = taskId;
}
@Override
public ListenableFuture<ServiceLocations> locate()
{
final Optional<TaskStatus> status = taskInfoProvider.getTaskStatus(taskId);
if (status.isPresent() && status.get().isRunnable()) {
final TaskLocation location = taskInfoProvider.getTaskLocation(taskId);
if (location.getHost() == null) {
return Futures.immediateFuture(ServiceLocations.forLocations(Collections.emptySet()));
} else {
return Futures.immediateFuture(
ServiceLocations.forLocation(
new ServiceLocation(
location.getHost(),
location.getPort(),
location.getTlsPort(),
StringUtils.format("%s/%s", BASE_PATH, StringUtils.urlEncode(taskId))
)
)
);
}
} else {
return Futures.immediateFuture(ServiceLocations.closed());
}
}
@Override
public void close()
{
// Nothing to do. Instance holds no state.
}
}
}