mirror of https://github.com/apache/druid.git
Async task client for SeekableStreamSupervisors. (#13354)
Main changes: 1) Convert SeekableStreamIndexTaskClient to an interface, move old code to SeekableStreamIndexTaskClientSyncImpl, and add new implementation SeekableStreamIndexTaskClientAsyncImpl that uses ServiceClient. 2) Add "chatAsync" parameter to seekable stream supervisors that causes the supervisor to use an async task client. 3) In SeekableStreamSupervisor.discoverTasks, adjust logic to avoid making blocking RPC calls in workerExec threads. 4) In SeekableStreamSupervisor generally, switch from Futures.successfulAsList to FutureUtils.coalesce, so we can better capture the errors that occurred with contacting individual tasks. Other, related changes: 1) Add ServiceRetryPolicy.retryNotAvailable, which controls whether ServiceClient retries unavailable services. Useful since we do not want to retry calls unavailable tasks within the service client. (The supervisor does its own higher-level retries.) 2) Add FutureUtils.transformAsync, a more lambda friendly version of Futures.transform(f, AsyncFunction). 3) Add FutureUtils.coalesce. Similar to Futures.successfulAsList, but returns Either instead of using null on error. 4) Add JacksonUtils.readValue overloads for JavaType and TypeReference.
This commit is contained in:
parent
f037776fd8
commit
bfffbabb56
|
@ -19,14 +19,18 @@
|
|||
|
||||
package org.apache.druid.common.guava;
|
||||
|
||||
import com.google.common.util.concurrent.AsyncFunction;
|
||||
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.SettableFuture;
|
||||
import org.apache.druid.java.util.common.Either;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.function.Function;
|
||||
|
||||
|
@ -106,6 +110,57 @@ public class FutureUtils
|
|||
return Futures.transform(future, fn::apply);
|
||||
}
|
||||
|
||||
/**
|
||||
* Like {@link Futures#transform(ListenableFuture, AsyncFunction)}, but works better with lambdas due to not having
|
||||
* overloads.
|
||||
*
|
||||
* One can write {@code FutureUtils.transformAsync(future, v -> ...)} instead of
|
||||
* {@code Futures.transform(future, (Function<? super T, ?>) v -> ...)}
|
||||
*/
|
||||
public static <T, R> ListenableFuture<R> transformAsync(final ListenableFuture<T> future, final AsyncFunction<T, R> fn)
|
||||
{
|
||||
return Futures.transform(future, fn);
|
||||
}
|
||||
|
||||
/**
|
||||
* Like {@link Futures#successfulAsList}, but returns {@link Either} instead of using {@code null} in case of error.
|
||||
*/
|
||||
public static <T> ListenableFuture<List<Either<Throwable, T>>> coalesce(final List<ListenableFuture<T>> futures)
|
||||
{
|
||||
return transform(
|
||||
Futures.successfulAsList(futures),
|
||||
values -> {
|
||||
final List<Either<Throwable, T>> eithers = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i < values.size(); i++) {
|
||||
final ListenableFuture<T> future = futures.get(i);
|
||||
final T value = values.get(i);
|
||||
|
||||
if (value != null) {
|
||||
eithers.add(Either.value(value));
|
||||
} else {
|
||||
try {
|
||||
future.get();
|
||||
}
|
||||
catch (ExecutionException e) {
|
||||
eithers.add(Either.error(e.getCause()));
|
||||
continue;
|
||||
}
|
||||
catch (Throwable e) {
|
||||
eithers.add(Either.error(e));
|
||||
continue;
|
||||
}
|
||||
|
||||
// No exception: value must really have been null.
|
||||
eithers.add(Either.value(null));
|
||||
}
|
||||
}
|
||||
|
||||
return eithers;
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a future that resolves when "future" resolves and "baggage" has been closed. If the baggage is closed
|
||||
* successfully, the returned future will have the same value (or exception status) as the input future. If the
|
||||
|
|
|
@ -62,6 +62,11 @@ public class Either<L, R>
|
|||
return error != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the error object.
|
||||
*
|
||||
* @throws IllegalStateException if this instance is not an error
|
||||
*/
|
||||
public L error()
|
||||
{
|
||||
if (isError()) {
|
||||
|
@ -74,11 +79,10 @@ public class Either<L, R>
|
|||
/**
|
||||
* If this Either represents a value, returns it. If this Either represents an error, throw an error.
|
||||
*
|
||||
* If the error is a {@link RuntimeException} or {@link Error}, it is thrown directly. If it is some other
|
||||
* {@link Throwable}, it is wrapped in a RuntimeException and thrown. If it is not a throwable at all, a generic
|
||||
* error is thrown containing the string representation of the error object.
|
||||
* If the error is a {@link Throwable}, it is wrapped in a RuntimeException and thrown. If it is not a throwable,
|
||||
* a generic error is thrown containing the string representation of the error object.
|
||||
*
|
||||
* If you want to be able to retrieve the error as-is, use {@link #isError()} and {@link #error()} instead.
|
||||
* To retrieve the error as-is, use {@link #isError()} and {@link #error()} instead.
|
||||
*/
|
||||
@Nullable
|
||||
public R valueOrThrow()
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.java.util.common.jackson;
|
|||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.JavaType;
|
||||
import com.fasterxml.jackson.databind.JsonMappingException;
|
||||
import com.fasterxml.jackson.databind.JsonSerializer;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
@ -54,10 +55,36 @@ public final class JacksonUtils
|
|||
/**
|
||||
* Silences Jackson's {@link IOException}.
|
||||
*/
|
||||
public static <T> T readValue(ObjectMapper mapper, byte[] bytes, Class<T> valueClass)
|
||||
public static <T> T readValue(ObjectMapper mapper, byte[] bytes, Class<T> valueType)
|
||||
{
|
||||
try {
|
||||
return mapper.readValue(bytes, valueClass);
|
||||
return mapper.readValue(bytes, valueType);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Silences Jackson's {@link IOException}.
|
||||
*/
|
||||
public static <T> T readValue(ObjectMapper mapper, byte[] bytes, JavaType valueType)
|
||||
{
|
||||
try {
|
||||
return mapper.readValue(bytes, valueType);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Silences Jackson's {@link IOException}.
|
||||
*/
|
||||
public static <T> T readValue(ObjectMapper mapper, byte[] bytes, TypeReference<T> valueType)
|
||||
{
|
||||
try {
|
||||
return mapper.readValue(bytes, valueType);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
|
|
|
@ -19,9 +19,11 @@
|
|||
|
||||
package org.apache.druid.common.guava;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.SettableFuture;
|
||||
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;
|
||||
|
@ -33,11 +35,15 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
import org.junit.internal.matchers.ThrowableMessageMatcher;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CancellationException;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
|
@ -200,6 +206,104 @@ public class FutureUtilsTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_transformAsync() throws Exception
|
||||
{
|
||||
Assert.assertEquals(
|
||||
"xy",
|
||||
FutureUtils.transformAsync(Futures.immediateFuture("x"), s -> Futures.immediateFuture(s + "y")).get()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_coalesce_allOk() throws Exception
|
||||
{
|
||||
final List<ListenableFuture<String>> futures = new ArrayList<>();
|
||||
|
||||
futures.add(Futures.immediateFuture("foo"));
|
||||
futures.add(Futures.immediateFuture("bar"));
|
||||
futures.add(Futures.immediateFuture(null));
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(Either.value("foo"), Either.value("bar"), Either.value(null)),
|
||||
FutureUtils.coalesce(futures).get()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_coalesce_inputError() throws Exception
|
||||
{
|
||||
final List<ListenableFuture<String>> futures = new ArrayList<>();
|
||||
|
||||
final ISE e = new ISE("oops");
|
||||
futures.add(Futures.immediateFuture("foo"));
|
||||
futures.add(Futures.immediateFailedFuture(e));
|
||||
futures.add(Futures.immediateFuture(null));
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(Either.value("foo"), Either.error(e), Either.value(null)),
|
||||
FutureUtils.coalesce(futures).get()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_coalesce_inputCanceled() throws Exception
|
||||
{
|
||||
final List<ListenableFuture<String>> futures = new ArrayList<>();
|
||||
|
||||
futures.add(Futures.immediateFuture("foo"));
|
||||
futures.add(Futures.immediateCancelledFuture());
|
||||
futures.add(Futures.immediateFuture(null));
|
||||
|
||||
final List<Either<Throwable, String>> results = FutureUtils.coalesce(futures).get();
|
||||
Assert.assertEquals(3, results.size());
|
||||
Assert.assertEquals(Either.value("foo"), results.get(0));
|
||||
Assert.assertTrue(results.get(1).isError());
|
||||
Assert.assertEquals(Either.value(null), results.get(2));
|
||||
|
||||
MatcherAssert.assertThat(
|
||||
results.get(1).error(),
|
||||
CoreMatchers.instanceOf(CancellationException.class)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_coalesce_timeout()
|
||||
{
|
||||
final List<ListenableFuture<String>> futures = new ArrayList<>();
|
||||
final SettableFuture<String> unresolvedFuture = SettableFuture.create();
|
||||
|
||||
futures.add(Futures.immediateFuture("foo"));
|
||||
futures.add(unresolvedFuture);
|
||||
futures.add(Futures.immediateFuture(null));
|
||||
|
||||
final ListenableFuture<List<Either<Throwable, String>>> coalesced = FutureUtils.coalesce(futures);
|
||||
|
||||
Assert.assertThrows(
|
||||
TimeoutException.class,
|
||||
() -> coalesced.get(10, TimeUnit.MILLISECONDS)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_coalesce_cancel()
|
||||
{
|
||||
final List<ListenableFuture<String>> futures = new ArrayList<>();
|
||||
final SettableFuture<String> unresolvedFuture = SettableFuture.create();
|
||||
|
||||
futures.add(Futures.immediateFuture("foo"));
|
||||
futures.add(unresolvedFuture);
|
||||
futures.add(Futures.immediateFuture(null));
|
||||
|
||||
final ListenableFuture<List<Either<Throwable, String>>> coalesced = FutureUtils.coalesce(futures);
|
||||
coalesced.cancel(true);
|
||||
|
||||
Assert.assertTrue(coalesced.isCancelled());
|
||||
|
||||
// All input futures are canceled too.
|
||||
Assert.assertTrue(unresolvedFuture.isCancelled());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_futureWithBaggage_ok() throws ExecutionException, InterruptedException
|
||||
{
|
||||
|
|
|
@ -25,6 +25,8 @@ import com.fasterxml.jackson.core.JsonGenerator;
|
|||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.SerializerProvider;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.jackson.JacksonUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -33,6 +35,7 @@ import java.io.ByteArrayOutputStream;
|
|||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class JacksonUtilsTest
|
||||
|
@ -91,6 +94,32 @@ public class JacksonUtilsTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadValue()
|
||||
{
|
||||
final ObjectMapper objectMapper = new ObjectMapper();
|
||||
final byte[] bytes = StringUtils.toUtf8("{\"foo\":\"bar\"}");
|
||||
final Map<String, Object> expected = ImmutableMap.of("foo", "bar");
|
||||
|
||||
Assert.assertEquals(
|
||||
"readValue(Class)",
|
||||
expected,
|
||||
JacksonUtils.readValue(objectMapper, bytes, Map.class)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"readValue(JavaType)",
|
||||
expected,
|
||||
JacksonUtils.readValue(objectMapper, bytes, objectMapper.constructType(Map.class))
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"readValue(TypeReference)",
|
||||
expected,
|
||||
JacksonUtils.readValue(objectMapper, bytes, JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING)
|
||||
);
|
||||
}
|
||||
|
||||
public static class SerializableClass
|
||||
{
|
||||
private final int value;
|
||||
|
|
|
@ -206,7 +206,8 @@ The `tuningConfig` is optional and default parameters will be used if no `tuning
|
|||
| `handoffConditionTimeout` | Long | Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever. | no (default == 0) |
|
||||
| `resetOffsetAutomatically` | Boolean | Controls behavior when Druid needs to read Kafka messages that are no longer available (i.e. when `OffsetOutOfRangeException` is encountered).<br/><br/>If false, the exception will bubble up, which will cause your tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation; potentially using the [Reset Supervisor API](../../operations/api-reference.md#supervisors). This mode is useful for production, since it will make you aware of issues with ingestion.<br/><br/>If true, Druid will automatically reset to the earlier or latest offset available in Kafka, based on the value of the `useEarliestOffset` property (earliest if true, latest if false). Note that this can lead to data being _DROPPED_ (if `useEarliestOffset` is false) or _DUPLICATED_ (if `useEarliestOffset` is true) without your knowledge. Messages will be logged indicating that a reset has occurred, but ingestion will continue. This mode is useful for non-production situations, since it will make Druid attempt to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.<br/><br/>This feature behaves similarly to the Kafka `auto.offset.reset` consumer property. | no (default == false) |
|
||||
| `workerThreads` | Integer | The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation. | no (default == min(10, taskCount)) |
|
||||
| `chatThreads` | Integer | The number of threads that will be used for communicating with indexing tasks. | no (default == min(10, taskCount * replicas)) |
|
||||
| `chatAsync` | Boolean | If true, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If false, use synchronous communication in a thread pool of size `chatThreads`. | no (default == false) |
|
||||
| `chatThreads` | Integer | The number of threads that will be used for communicating with indexing tasks. Ignored if `chatAsync` is `true`. | no (default == min(10, taskCount * replicas)) |
|
||||
| `chatRetries` | Integer | The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive. | no (default == 8) |
|
||||
| `httpTimeout` | ISO8601 Period | How long to wait for a HTTP response from an indexing task. | no (default == PT10S) |
|
||||
| `shutdownTimeout` | ISO8601 Period | How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting. | no (default == PT80S) |
|
||||
|
|
|
@ -293,7 +293,8 @@ The `tuningConfig` is optional. If no `tuningConfig` is specified, default param
|
|||
|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kinesis messages that are no longer available.<br/><br/>If false, the exception will bubble up, which will cause your tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation; potentially using the [Reset Supervisor API](../../operations/api-reference.md#supervisors). This mode is useful for production, since it will make you aware of issues with ingestion.<br/><br/>If true, Druid will automatically reset to the earlier or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if true, latest if false). Please note that this can lead to data being _DROPPED_ (if `useEarliestSequenceNumber` is false) or _DUPLICATED_ (if `useEarliestSequenceNumber` is true) without your knowledge. Messages will be logged indicating that a reset has occurred, but ingestion will continue. This mode is useful for non-production situations, since it will make Druid attempt to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.|no (default == false)|
|
||||
|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If set to false, the indexing task will attempt to reset the current sequence number (or not), depending on the value of `resetOffsetAutomatically`.|no (default == false)|
|
||||
|`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|no (default == min(10, taskCount))|
|
||||
|`chatThreads`|Integer|The number of threads that will be used for communicating with indexing tasks.|no (default == min(10, taskCount * replicas))|
|
||||
|`chatAsync`|Boolean| If true, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If false, use synchronous communication in a thread pool of size `chatThreads`. | no (default == false) |
|
||||
|`chatThreads`|Integer| The number of threads that will be used for communicating with indexing tasks. Ignored if `chatAsync` is `true`.| no (default == min(10, taskCount * replicas))|
|
||||
|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive.| no (default == 8)|
|
||||
|`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)|
|
||||
|`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)|
|
||||
|
|
|
@ -1,62 +0,0 @@
|
|||
/*
|
||||
* 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.kafka;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
public class KafkaIndexTaskClient extends SeekableStreamIndexTaskClient<Integer, Long>
|
||||
{
|
||||
KafkaIndexTaskClient(
|
||||
HttpClient httpClient,
|
||||
ObjectMapper jsonMapper,
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
)
|
||||
{
|
||||
super(
|
||||
httpClient,
|
||||
jsonMapper,
|
||||
taskInfoProvider,
|
||||
dataSource,
|
||||
numThreads,
|
||||
httpTimeout,
|
||||
numRetries
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<Integer> getPartitionType()
|
||||
{
|
||||
return Integer.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<Long> getSequenceType()
|
||||
{
|
||||
return Long.class;
|
||||
}
|
||||
}
|
|
@ -21,41 +21,35 @@ package org.apache.druid.indexing.kafka;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
import org.joda.time.Duration;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
|
||||
public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<KafkaIndexTaskClient>
|
||||
@LazySingleton
|
||||
public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<Integer, Long>
|
||||
{
|
||||
@Inject
|
||||
public KafkaIndexTaskClientFactory(
|
||||
@EscalatedGlobal ServiceClientFactory serviceClientFactory,
|
||||
@EscalatedGlobal HttpClient httpClient,
|
||||
@Json ObjectMapper mapper
|
||||
)
|
||||
{
|
||||
super(httpClient, mapper);
|
||||
super(serviceClientFactory, httpClient, mapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
public KafkaIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
)
|
||||
public Class<Integer> getPartitionType()
|
||||
{
|
||||
return new KafkaIndexTaskClient(
|
||||
getHttpClient(),
|
||||
getMapper(),
|
||||
taskInfoProvider,
|
||||
dataSource,
|
||||
numThreads,
|
||||
httpTimeout,
|
||||
numRetries
|
||||
);
|
||||
return Integer.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<Long> getSequenceType()
|
||||
{
|
||||
return Long.class;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,12 +24,9 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
|
|||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.TypeLiteral;
|
||||
import org.apache.druid.data.input.kafkainput.KafkaInputFormat;
|
||||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec;
|
||||
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -59,10 +56,6 @@ public class KafkaIndexTaskModule implements DruidModule
|
|||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
binder.bind(
|
||||
new TypeLiteral<SeekableStreamIndexTaskClientFactory<KafkaIndexTaskClient>>()
|
||||
{
|
||||
}
|
||||
).to(KafkaIndexTaskClientFactory.class).in(LazySingleton.class);
|
||||
// Nothing to do.
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.indexing.kafka.supervisor;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
|
||||
|
@ -34,6 +35,7 @@ public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig
|
|||
implements SeekableStreamSupervisorTuningConfig
|
||||
{
|
||||
private final Integer workerThreads;
|
||||
private final Boolean chatAsync;
|
||||
private final Integer chatThreads;
|
||||
private final Long chatRetries;
|
||||
private final Duration httpTimeout;
|
||||
|
@ -66,6 +68,7 @@ public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
@ -86,6 +89,7 @@ public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig
|
|||
@JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically,
|
||||
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
|
||||
@JsonProperty("workerThreads") Integer workerThreads,
|
||||
@JsonProperty("chatAsync") Boolean chatAsync,
|
||||
@JsonProperty("chatThreads") Integer chatThreads,
|
||||
@JsonProperty("chatRetries") Long chatRetries,
|
||||
@JsonProperty("httpTimeout") Period httpTimeout,
|
||||
|
@ -119,6 +123,7 @@ public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig
|
|||
maxSavedParseExceptions
|
||||
);
|
||||
this.workerThreads = workerThreads;
|
||||
this.chatAsync = chatAsync;
|
||||
this.chatThreads = chatThreads;
|
||||
this.chatRetries = (chatRetries != null ? chatRetries : DEFAULT_CHAT_RETRIES);
|
||||
this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, DEFAULT_HTTP_TIMEOUT);
|
||||
|
@ -139,6 +144,23 @@ public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig
|
|||
return workerThreads;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getChatAsync()
|
||||
{
|
||||
if (chatAsync != null) {
|
||||
return chatAsync;
|
||||
} else {
|
||||
return DEFAULT_ASYNC;
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty("chatAsync")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
Boolean getChatAsyncConfigured()
|
||||
{
|
||||
return chatAsync;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public Integer getChatThreads()
|
||||
|
|
|
@ -30,6 +30,8 @@ import org.apache.druid.indexer.TaskLocation;
|
|||
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.seekablestream.SeekableStreamIndexTaskClient;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientSyncImpl;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
@ -55,6 +57,8 @@ import org.junit.Assert;
|
|||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.internal.matchers.ThrowableCauseMatcher;
|
||||
import org.junit.internal.matchers.ThrowableMessageMatcher;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -66,6 +70,7 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class KafkaIndexTaskClientTest extends EasyMockSupport
|
||||
|
@ -90,7 +95,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
private StringFullResponseHolder responseHolder;
|
||||
private HttpResponse response;
|
||||
private HttpHeaders headers;
|
||||
private KafkaIndexTaskClient client;
|
||||
private SeekableStreamIndexTaskClient<Integer, Long> client;
|
||||
|
||||
@Parameterized.Parameters(name = "numThreads = {0}")
|
||||
public static Iterable<Object[]> constructorFeeder()
|
||||
|
@ -137,7 +142,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testNoTaskLocation() throws IOException
|
||||
public void testNoTaskLocation() throws Exception
|
||||
{
|
||||
EasyMock.reset(taskInfoProvider);
|
||||
EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes();
|
||||
|
@ -146,25 +151,27 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
.anyTimes();
|
||||
replayAll();
|
||||
|
||||
Assert.assertFalse(client.stop(TEST_ID, true));
|
||||
Assert.assertFalse(client.resume(TEST_ID));
|
||||
Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID));
|
||||
Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID));
|
||||
Assert.assertEquals(Status.NOT_STARTED, client.getStatus(TEST_ID));
|
||||
Assert.assertNull(client.getStartTime(TEST_ID));
|
||||
Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true));
|
||||
Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID));
|
||||
Assert.assertFalse(client.setEndOffsets(TEST_ID, Collections.emptyMap(), true));
|
||||
Assert.assertFalse(client.setEndOffsets(TEST_ID, Collections.emptyMap(), true));
|
||||
Assert.assertFalse(client.stopAsync(TEST_ID, true).get());
|
||||
Assert.assertFalse(client.resumeAsync(TEST_ID).get());
|
||||
Assert.assertEquals(ImmutableMap.of(), client.pauseAsync(TEST_ID).get());
|
||||
Assert.assertEquals(ImmutableMap.of(), client.pauseAsync(TEST_ID).get());
|
||||
Assert.assertEquals(Status.NOT_STARTED, client.getStatusAsync(TEST_ID).get());
|
||||
Assert.assertNull(client.getStartTimeAsync(TEST_ID).get());
|
||||
Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsetsAsync(TEST_ID, true).get());
|
||||
Assert.assertEquals(ImmutableMap.of(), client.getEndOffsetsAsync(TEST_ID).get());
|
||||
Assert.assertFalse(client.setEndOffsetsAsync(TEST_ID, Collections.emptyMap(), true).get());
|
||||
Assert.assertFalse(client.setEndOffsetsAsync(TEST_ID, Collections.emptyMap(), true).get());
|
||||
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskNotRunnableException()
|
||||
public void testTaskNotRunnableException() throws Exception
|
||||
{
|
||||
expectedException.expect(IndexTaskClient.TaskNotRunnableException.class);
|
||||
expectedException.expectMessage("Aborting request because task [test-id] is not runnable");
|
||||
expectedException.expect(ExecutionException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(IndexTaskClient.TaskNotRunnableException.class));
|
||||
expectedException.expectCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
|
||||
"Aborting request because task [test-id] is not runnable")));
|
||||
|
||||
EasyMock.reset(taskInfoProvider);
|
||||
EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID))
|
||||
|
@ -175,16 +182,17 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
.anyTimes();
|
||||
replayAll();
|
||||
|
||||
client.getCurrentOffsets(TEST_ID, true);
|
||||
client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInternalServerError()
|
||||
public void testInternalServerError() throws Exception
|
||||
{
|
||||
expectedException.expect(RuntimeException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(IOException.class));
|
||||
expectedException.expectMessage("Received server error with status [500 Internal Server Error]");
|
||||
expectedException.expect(ExecutionException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(RuntimeException.class));
|
||||
expectedException.expectCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
|
||||
"Received server error with status [500 Internal Server Error]")));
|
||||
|
||||
EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2);
|
||||
EasyMock.expect(responseHolder.getContent()).andReturn("");
|
||||
|
@ -199,15 +207,17 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.getCurrentOffsets(TEST_ID, true);
|
||||
client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBadRequest()
|
||||
public void testBadRequest() throws Exception
|
||||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Received server error with status [400 Bad Request]");
|
||||
expectedException.expect(ExecutionException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(IllegalArgumentException.class));
|
||||
expectedException.expectCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
|
||||
"Received server error with status [400 Bad Request]")));
|
||||
|
||||
EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2);
|
||||
EasyMock.expect(responseHolder.getContent()).andReturn("");
|
||||
|
@ -222,12 +232,12 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.getCurrentOffsets(TEST_ID, true);
|
||||
client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskLocationMismatch()
|
||||
public void testTaskLocationMismatch() throws Exception
|
||||
{
|
||||
EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(2);
|
||||
EasyMock.expect(responseHolder.getResponse()).andReturn(response);
|
||||
|
@ -247,7 +257,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Map<Integer, Long> results = client.getCurrentOffsets(TEST_ID, true);
|
||||
Map<Integer, Long> results = client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(0, results.size());
|
||||
|
@ -267,7 +277,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Map<Integer, Long> results = client.getCurrentOffsets(TEST_ID, true);
|
||||
Map<Integer, Long> results = client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -308,7 +318,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
|
||||
replayAll();
|
||||
|
||||
Map<Integer, Long> results = client.getCurrentOffsets(TEST_ID, true);
|
||||
Map<Integer, Long> results = client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(3, captured.getValues().size());
|
||||
|
@ -327,11 +337,13 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetCurrentOffsetsWithExhaustedRetries()
|
||||
public void testGetCurrentOffsetsWithExhaustedRetries() throws Exception
|
||||
{
|
||||
expectedException.expect(RuntimeException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(IOException.class));
|
||||
expectedException.expectMessage("Received server error with status [404 Not Found]");
|
||||
expectedException.expect(ExecutionException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(RuntimeException.class));
|
||||
expectedException.expectCause(ThrowableCauseMatcher.hasCause(CoreMatchers.instanceOf(IOException.class)));
|
||||
expectedException.expectCause(ThrowableCauseMatcher.hasCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
|
||||
"Received server error with status [404 Not Found]"))));
|
||||
|
||||
client = new TestableKafkaIndexTaskClient(httpClient, OBJECT_MAPPER, taskInfoProvider, 2);
|
||||
|
||||
|
@ -350,7 +362,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).andReturn(errorResponseHolder()).anyTimes();
|
||||
replayAll();
|
||||
|
||||
client.getCurrentOffsets(TEST_ID, true);
|
||||
client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
|
@ -368,7 +380,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Map<Integer, Long> results = client.getEndOffsets(TEST_ID);
|
||||
Map<Integer, Long> results = client.getEndOffsetsAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -403,7 +415,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
)).andReturn(errorResponseHolder()).once().andReturn(okResponseHolder());
|
||||
replayAll();
|
||||
|
||||
DateTime results = client.getStartTime(TEST_ID);
|
||||
DateTime results = client.getStartTimeAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -433,7 +445,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Status results = client.getStatus(TEST_ID);
|
||||
Status results = client.getStatusAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -462,7 +474,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Map<Integer, Long> results = client.pause(TEST_ID);
|
||||
Map<Integer, Long> results = client.pauseAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -496,10 +508,14 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
replayAll();
|
||||
|
||||
try {
|
||||
client.pause(TEST_ID);
|
||||
client.pauseAsync(TEST_ID).get();
|
||||
}
|
||||
catch (Exception ex) {
|
||||
Assert.assertEquals("Task [test-id] failed to change its status from [READING] to [PAUSED], aborting", ex.getMessage());
|
||||
Assert.assertEquals(
|
||||
"org.apache.druid.java.util.common.ISE: "
|
||||
+ "Task [test-id] failed to change its status from [READING] to [PAUSED], aborting",
|
||||
ex.getMessage()
|
||||
);
|
||||
verifyAll();
|
||||
return;
|
||||
}
|
||||
|
@ -540,7 +556,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
|
||||
replayAll();
|
||||
|
||||
Map<Integer, Long> results = client.pause(TEST_ID);
|
||||
Map<Integer, Long> results = client.pauseAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -584,7 +600,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.resume(TEST_ID);
|
||||
client.resumeAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -612,7 +628,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.setEndOffsets(TEST_ID, endOffsets, true);
|
||||
client.setEndOffsetsAsync(TEST_ID, endOffsets, true).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -641,7 +657,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.setEndOffsets(TEST_ID, endOffsets, true);
|
||||
client.setEndOffsetsAsync(TEST_ID, endOffsets, true).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -668,7 +684,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.stop(TEST_ID, false);
|
||||
client.stopAsync(TEST_ID, false).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -694,7 +710,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.stop(TEST_ID, true);
|
||||
client.stopAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -1064,7 +1080,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
return Futures.immediateFuture(Either.error(responseHolder));
|
||||
}
|
||||
|
||||
private class TestableKafkaIndexTaskClient extends KafkaIndexTaskClient
|
||||
private class TestableKafkaIndexTaskClient extends SeekableStreamIndexTaskClientSyncImpl<Integer, Long>
|
||||
{
|
||||
TestableKafkaIndexTaskClient(
|
||||
HttpClient httpClient,
|
||||
|
@ -1088,6 +1104,19 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
@Override
|
||||
protected void checkConnection(String host, int port)
|
||||
{
|
||||
// Do nothing.
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<Integer> getPartitionType()
|
||||
{
|
||||
return Integer.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<Long> getSequenceType()
|
||||
{
|
||||
return Long.class;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -143,6 +143,7 @@ public class KafkaIndexTaskTuningConfigTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
KafkaIndexTaskTuningConfig copy = original.convertToTaskTuningConfig();
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.druid.indexing.common.task.Task;
|
|||
import org.apache.druid.indexing.kafka.KafkaConsumerConfigs;
|
||||
import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata;
|
||||
import org.apache.druid.indexing.kafka.KafkaIndexTask;
|
||||
import org.apache.druid.indexing.kafka.KafkaIndexTaskClient;
|
||||
import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory;
|
||||
import org.apache.druid.indexing.kafka.KafkaIndexTaskIOConfig;
|
||||
import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig;
|
||||
|
@ -63,6 +62,7 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
|
|||
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig;
|
||||
import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
|
||||
|
@ -70,6 +70,7 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
|
|||
import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.LagBasedAutoScalerConfig;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
@ -104,7 +105,6 @@ import org.easymock.EasyMock;
|
|||
import org.easymock.EasyMockSupport;
|
||||
import org.easymock.IAnswer;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -158,7 +158,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
private TaskMaster taskMaster;
|
||||
private TaskRunner taskRunner;
|
||||
private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
|
||||
private KafkaIndexTaskClient taskClient;
|
||||
private SeekableStreamIndexTaskClient<Integer, Long> taskClient;
|
||||
private TaskQueue taskQueue;
|
||||
private String topic;
|
||||
private RowIngestionMetersFactory rowIngestionMetersFactory;
|
||||
|
@ -213,7 +213,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
taskMaster = createMock(TaskMaster.class);
|
||||
taskRunner = createMock(TaskRunner.class);
|
||||
indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class);
|
||||
taskClient = createMock(KafkaIndexTaskClient.class);
|
||||
taskClient = createMock(SeekableStreamIndexTaskClient.class);
|
||||
EasyMock.expect(taskClient.resumeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(true)).anyTimes();
|
||||
taskQueue = createMock(TaskQueue.class);
|
||||
|
||||
|
@ -244,23 +244,26 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
@Test
|
||||
public void testNoInitialStateWithAutoscaler() throws Exception
|
||||
{
|
||||
final int taskCountMax = 2;
|
||||
final int replicas = 1;
|
||||
|
||||
KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory(
|
||||
null,
|
||||
null
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public KafkaIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
public SeekableStreamIndexTaskClient<Integer, Long> build(
|
||||
final String dataSource,
|
||||
final TaskInfoProvider taskInfoProvider,
|
||||
final int maxNumTasks,
|
||||
final SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(TEST_CHAT_THREADS, numThreads);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout);
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, numRetries);
|
||||
Assert.assertEquals(replicas * taskCountMax, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
}
|
||||
};
|
||||
|
@ -275,7 +278,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
autoScalerConfig.put("triggerScaleInFractionThreshold", 0.8);
|
||||
autoScalerConfig.put("scaleActionStartDelayMillis", 0);
|
||||
autoScalerConfig.put("scaleActionPeriodMillis", 600);
|
||||
autoScalerConfig.put("taskCountMax", 2);
|
||||
autoScalerConfig.put("taskCountMax", taskCountMax);
|
||||
autoScalerConfig.put("taskCountMin", 1);
|
||||
autoScalerConfig.put("scaleInStep", 1);
|
||||
autoScalerConfig.put("scaleOutStep", 2);
|
||||
|
@ -288,7 +291,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig(
|
||||
topic,
|
||||
INPUT_FORMAT,
|
||||
1,
|
||||
replicas,
|
||||
1,
|
||||
new Period("PT1H"),
|
||||
consumerProperties,
|
||||
|
@ -321,6 +324,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
false,
|
||||
null,
|
||||
numThreads,
|
||||
null,
|
||||
TEST_CHAT_THREADS,
|
||||
TEST_CHAT_RETRIES,
|
||||
TEST_HTTP_TIMEOUT,
|
||||
|
@ -1552,13 +1556,14 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING));
|
||||
EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1"))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)));
|
||||
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true);
|
||||
|
||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||
checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L));
|
||||
EasyMock.expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean()))
|
||||
.andReturn(checkpoints)
|
||||
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean()))
|
||||
.andReturn(Futures.immediateFuture(checkpoints))
|
||||
.anyTimes();
|
||||
|
||||
taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
|
||||
|
@ -1668,7 +1673,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING));
|
||||
EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L, 2, 30L)));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L));
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1"))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L, 2, 30L)));
|
||||
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true);
|
||||
|
||||
taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
|
||||
|
@ -1803,7 +1809,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L));
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1"))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)));
|
||||
EasyMock.expect(taskClient.getCurrentOffsetsAsync("id2", false))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 4L, 1, 5L, 2, 6L)));
|
||||
|
||||
|
@ -2810,7 +2817,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1"))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)));
|
||||
|
||||
// getCheckpoints will not be called for id1 as it is in publishing state
|
||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||
|
@ -3077,7 +3085,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1"))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)));
|
||||
|
||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||
checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||
|
@ -3534,7 +3543,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
.andReturn(Futures.immediateFuture(Status.READING));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1"))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)));
|
||||
|
||||
// getCheckpoints will not be called for id1 as it is in publishing state
|
||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||
|
@ -3974,6 +3984,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
false,
|
||||
null,
|
||||
numThreads,
|
||||
null,
|
||||
TEST_CHAT_THREADS,
|
||||
TEST_CHAT_RETRIES,
|
||||
TEST_HTTP_TIMEOUT,
|
||||
|
@ -4013,6 +4024,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
null,
|
||||
null,
|
||||
numThreads,
|
||||
null,
|
||||
TEST_CHAT_THREADS,
|
||||
TEST_CHAT_RETRIES,
|
||||
TEST_HTTP_TIMEOUT,
|
||||
|
@ -4234,7 +4246,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
EasyMock.expect(taskClient.getStatusAsync(failsToResumePausedTask.getId()))
|
||||
.andReturn(Futures.immediateFuture(Status.PAUSED));
|
||||
|
||||
EasyMock.expect(taskClient.getEndOffsets(publishingTask.getId())).andReturn(ImmutableMap.of(0, 0L));
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync(publishingTask.getId()))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 0L)));
|
||||
|
||||
EasyMock.expect(taskClient.getCheckpointsAsync(readingTask.getId(), true))
|
||||
.andReturn(Futures.immediateFuture(new TreeMap<>())).anyTimes();
|
||||
|
@ -4459,22 +4472,22 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
);
|
||||
|
||||
KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory(
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public KafkaIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
public SeekableStreamIndexTaskClient<Integer, Long> build(
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(TEST_CHAT_THREADS, numThreads);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout);
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, numRetries);
|
||||
Assert.assertEquals(replicas * taskCount, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
}
|
||||
};
|
||||
|
@ -4495,6 +4508,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
resetOffsetAutomatically,
|
||||
null,
|
||||
numThreads,
|
||||
null,
|
||||
TEST_CHAT_THREADS,
|
||||
TEST_CHAT_RETRIES,
|
||||
TEST_HTTP_TIMEOUT,
|
||||
|
@ -4572,22 +4586,22 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
);
|
||||
|
||||
KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory(
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public KafkaIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
public SeekableStreamIndexTaskClient<Integer, Long> build(
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(TEST_CHAT_THREADS, numThreads);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout);
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, numRetries);
|
||||
Assert.assertEquals(replicas * taskCount, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
}
|
||||
};
|
||||
|
@ -4608,6 +4622,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
false,
|
||||
null,
|
||||
numThreads,
|
||||
null,
|
||||
TEST_CHAT_THREADS,
|
||||
TEST_CHAT_RETRIES,
|
||||
TEST_HTTP_TIMEOUT,
|
||||
|
@ -4689,22 +4704,22 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
);
|
||||
|
||||
KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory(
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public KafkaIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
public SeekableStreamIndexTaskClient<Integer, Long> build(
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(TEST_CHAT_THREADS, numThreads);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout);
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, numRetries);
|
||||
Assert.assertEquals(replicas * taskCount, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -1,64 +0,0 @@
|
|||
/*
|
||||
* 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.kinesis;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
public class KinesisIndexTaskClient extends SeekableStreamIndexTaskClient<String, String>
|
||||
{
|
||||
KinesisIndexTaskClient(
|
||||
HttpClient httpClient,
|
||||
ObjectMapper jsonMapper,
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
)
|
||||
{
|
||||
super(
|
||||
httpClient,
|
||||
jsonMapper,
|
||||
taskInfoProvider,
|
||||
dataSource,
|
||||
numThreads,
|
||||
httpTimeout,
|
||||
numRetries
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<String> getPartitionType()
|
||||
{
|
||||
return String.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<String> getSequenceType()
|
||||
{
|
||||
return String.class;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -21,41 +21,35 @@ package org.apache.druid.indexing.kinesis;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
import org.joda.time.Duration;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
|
||||
public class KinesisIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<KinesisIndexTaskClient>
|
||||
@LazySingleton
|
||||
public class KinesisIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<String, String>
|
||||
{
|
||||
@Inject
|
||||
public KinesisIndexTaskClientFactory(
|
||||
@EscalatedGlobal ServiceClientFactory serviceClientFactory,
|
||||
@EscalatedGlobal HttpClient httpClient,
|
||||
@Json ObjectMapper mapper
|
||||
)
|
||||
{
|
||||
super(httpClient, mapper);
|
||||
super(serviceClientFactory, httpClient, mapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
public KinesisIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
)
|
||||
public Class<String> getPartitionType()
|
||||
{
|
||||
return new KinesisIndexTaskClient(
|
||||
getHttpClient(),
|
||||
getMapper(),
|
||||
taskInfoProvider,
|
||||
dataSource,
|
||||
numThreads,
|
||||
httpTimeout,
|
||||
numRetries
|
||||
);
|
||||
return String.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<String> getSequenceType()
|
||||
{
|
||||
return String.class;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,14 +24,11 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
|
|||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.TypeLiteral;
|
||||
import com.google.inject.name.Names;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.guice.JsonConfigProvider;
|
||||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -60,12 +57,6 @@ public class KinesisIndexingServiceModule implements DruidModule
|
|||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
binder.bind(
|
||||
new TypeLiteral<SeekableStreamIndexTaskClientFactory<KinesisIndexTaskClient>>()
|
||||
{
|
||||
}
|
||||
).to(KinesisIndexTaskClientFactory.class).in(LazySingleton.class);
|
||||
|
||||
JsonConfigProvider.bind(binder, PROPERTY_BASE, AWSCredentialsConfig.class, Names.named(AWS_SCOPE));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.indexing.kinesis.supervisor;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
|
||||
|
@ -34,6 +35,7 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig
|
|||
implements SeekableStreamSupervisorTuningConfig
|
||||
{
|
||||
private final Integer workerThreads;
|
||||
private final Boolean chatAsync;
|
||||
private final Integer chatThreads;
|
||||
private final Long chatRetries;
|
||||
private final Duration httpTimeout;
|
||||
|
@ -76,6 +78,7 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
@ -97,6 +100,7 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig
|
|||
@JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck,
|
||||
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
|
||||
@JsonProperty("workerThreads") Integer workerThreads,
|
||||
@JsonProperty("chatAsync") Boolean chatAsync,
|
||||
@JsonProperty("chatThreads") Integer chatThreads,
|
||||
@JsonProperty("chatRetries") Long chatRetries,
|
||||
@JsonProperty("httpTimeout") Period httpTimeout,
|
||||
|
@ -144,6 +148,7 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig
|
|||
);
|
||||
|
||||
this.workerThreads = workerThreads;
|
||||
this.chatAsync = chatAsync;
|
||||
this.chatThreads = chatThreads;
|
||||
this.chatRetries = (chatRetries != null ? chatRetries : DEFAULT_CHAT_RETRIES);
|
||||
this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, DEFAULT_HTTP_TIMEOUT);
|
||||
|
@ -169,6 +174,23 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig
|
|||
return workerThreads;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getChatAsync()
|
||||
{
|
||||
if (chatAsync != null) {
|
||||
return chatAsync;
|
||||
} else {
|
||||
return DEFAULT_ASYNC;
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty("chatAsync")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
Boolean getChatAsyncConfigured()
|
||||
{
|
||||
return chatAsync;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public Integer getChatThreads()
|
||||
|
|
|
@ -30,6 +30,8 @@ import org.apache.druid.indexer.TaskLocation;
|
|||
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.seekablestream.SeekableStreamIndexTaskClient;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientSyncImpl;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
@ -55,6 +57,8 @@ import org.junit.Assert;
|
|||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.internal.matchers.ThrowableCauseMatcher;
|
||||
import org.junit.internal.matchers.ThrowableMessageMatcher;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -66,6 +70,7 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
|
@ -91,7 +96,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
private StringFullResponseHolder responseHolder;
|
||||
private HttpResponse response;
|
||||
private HttpHeaders headers;
|
||||
private KinesisIndexTaskClient client;
|
||||
private SeekableStreamIndexTaskClient<String, String> client;
|
||||
|
||||
@Parameterized.Parameters(name = "numThreads = {0}")
|
||||
public static Iterable<Object[]> constructorFeeder()
|
||||
|
@ -138,7 +143,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testNoTaskLocation() throws IOException
|
||||
public void testNoTaskLocation() throws Exception
|
||||
{
|
||||
EasyMock.reset(taskInfoProvider);
|
||||
EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes();
|
||||
|
@ -147,25 +152,27 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
.anyTimes();
|
||||
replayAll();
|
||||
|
||||
Assert.assertFalse(client.stop(TEST_ID, true));
|
||||
Assert.assertFalse(client.resume(TEST_ID));
|
||||
Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID));
|
||||
Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID));
|
||||
Assert.assertEquals(Status.NOT_STARTED, client.getStatus(TEST_ID));
|
||||
Assert.assertNull(client.getStartTime(TEST_ID));
|
||||
Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true));
|
||||
Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID));
|
||||
Assert.assertFalse(client.setEndOffsets(TEST_ID, Collections.emptyMap(), true));
|
||||
Assert.assertFalse(client.setEndOffsets(TEST_ID, Collections.emptyMap(), true));
|
||||
Assert.assertFalse(client.stopAsync(TEST_ID, true).get());
|
||||
Assert.assertFalse(client.resumeAsync(TEST_ID).get());
|
||||
Assert.assertEquals(ImmutableMap.of(), client.pauseAsync(TEST_ID).get());
|
||||
Assert.assertEquals(ImmutableMap.of(), client.pauseAsync(TEST_ID).get());
|
||||
Assert.assertEquals(Status.NOT_STARTED, client.getStatusAsync(TEST_ID).get());
|
||||
Assert.assertNull(client.getStartTimeAsync(TEST_ID).get());
|
||||
Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsetsAsync(TEST_ID, true).get());
|
||||
Assert.assertEquals(ImmutableMap.of(), client.getEndOffsetsAsync(TEST_ID).get());
|
||||
Assert.assertFalse(client.setEndOffsetsAsync(TEST_ID, Collections.emptyMap(), true).get());
|
||||
Assert.assertFalse(client.setEndOffsetsAsync(TEST_ID, Collections.emptyMap(), true).get());
|
||||
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskNotRunnableException()
|
||||
public void testTaskNotRunnableException() throws Exception
|
||||
{
|
||||
expectedException.expect(IndexTaskClient.TaskNotRunnableException.class);
|
||||
expectedException.expectMessage("Aborting request because task [test-id] is not runnable");
|
||||
expectedException.expect(ExecutionException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(IndexTaskClient.TaskNotRunnableException.class));
|
||||
expectedException.expectCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
|
||||
"Aborting request because task [test-id] is not runnable")));
|
||||
|
||||
EasyMock.reset(taskInfoProvider);
|
||||
EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID))
|
||||
|
@ -176,16 +183,17 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
.anyTimes();
|
||||
replayAll();
|
||||
|
||||
client.getCurrentOffsets(TEST_ID, true);
|
||||
client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInternalServerError()
|
||||
public void testInternalServerError() throws Exception
|
||||
{
|
||||
expectedException.expect(RuntimeException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(IOException.class));
|
||||
expectedException.expectMessage("Received server error with status [500 Internal Server Error]");
|
||||
expectedException.expect(ExecutionException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(RuntimeException.class));
|
||||
expectedException.expectCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
|
||||
"Received server error with status [500 Internal Server Error]")));
|
||||
|
||||
EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2);
|
||||
EasyMock.expect(responseHolder.getContent()).andReturn("");
|
||||
|
@ -200,15 +208,17 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.getCurrentOffsets(TEST_ID, true);
|
||||
client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBadRequest()
|
||||
public void testBadRequest() throws Exception
|
||||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Received server error with status [400 Bad Request]");
|
||||
expectedException.expect(ExecutionException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(IllegalArgumentException.class));
|
||||
expectedException.expectCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
|
||||
"Received server error with status [400 Bad Request]")));
|
||||
|
||||
EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2);
|
||||
EasyMock.expect(responseHolder.getContent()).andReturn("");
|
||||
|
@ -223,12 +233,12 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.getCurrentOffsets(TEST_ID, true);
|
||||
client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskLocationMismatch()
|
||||
public void testTaskLocationMismatch() throws Exception
|
||||
{
|
||||
EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(2);
|
||||
EasyMock.expect(responseHolder.getResponse()).andReturn(response);
|
||||
|
@ -248,7 +258,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Map<String, String> results = client.getCurrentOffsets(TEST_ID, true);
|
||||
Map<String, String> results = client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(0, results.size());
|
||||
|
@ -268,7 +278,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Map<String, String> results = client.getCurrentOffsets(TEST_ID, true);
|
||||
Map<String, String> results = client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -309,7 +319,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
|
||||
replayAll();
|
||||
|
||||
Map<String, String> results = client.getCurrentOffsets(TEST_ID, true);
|
||||
Map<String, String> results = client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(3, captured.getValues().size());
|
||||
|
@ -328,11 +338,13 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetCurrentOffsetsWithExhaustedRetries()
|
||||
public void testGetCurrentOffsetsWithExhaustedRetries() throws Exception
|
||||
{
|
||||
expectedException.expect(RuntimeException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(IOException.class));
|
||||
expectedException.expectMessage("Received server error with status [404 Not Found]");
|
||||
expectedException.expect(ExecutionException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(RuntimeException.class));
|
||||
expectedException.expectCause(ThrowableCauseMatcher.hasCause(CoreMatchers.instanceOf(IOException.class)));
|
||||
expectedException.expectCause(ThrowableCauseMatcher.hasCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
|
||||
"Received server error with status [404 Not Found]"))));
|
||||
|
||||
client = new TestableKinesisIndexTaskClient(httpClient, OBJECT_MAPPER, taskInfoProvider, 2);
|
||||
|
||||
|
@ -351,7 +363,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
).andReturn(errorResponseHolder()).anyTimes();
|
||||
replayAll();
|
||||
|
||||
client.getCurrentOffsets(TEST_ID, true);
|
||||
client.getCurrentOffsetsAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
}
|
||||
|
||||
|
@ -369,7 +381,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Map<String, String> results = client.getEndOffsets(TEST_ID);
|
||||
Map<String, String> results = client.getEndOffsetsAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -404,7 +416,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
)).andReturn(errorResponseHolder()).once().andReturn(okResponseHolder());
|
||||
replayAll();
|
||||
|
||||
DateTime results = client.getStartTime(TEST_ID);
|
||||
DateTime results = client.getStartTimeAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -434,7 +446,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Status results = client.getStatus(TEST_ID);
|
||||
Status results = client.getStatusAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -463,7 +475,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
Map<String, String> results = client.pause(TEST_ID);
|
||||
Map<String, String> results = client.pauseAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -512,7 +524,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
|
||||
replayAll();
|
||||
|
||||
Map<String, String> results = client.pause(TEST_ID);
|
||||
Map<String, String> results = client.pauseAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -560,10 +572,12 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
replayAll();
|
||||
|
||||
try {
|
||||
client.pause(TEST_ID);
|
||||
client.pauseAsync(TEST_ID).get();
|
||||
}
|
||||
catch (Exception ex) {
|
||||
Assert.assertEquals("Task [test-id] failed to change its status from [READING] to [PAUSED], aborting", ex.getMessage());
|
||||
Assert.assertEquals(
|
||||
"org.apache.druid.java.util.common.ISE: "
|
||||
+ "Task [test-id] failed to change its status from [READING] to [PAUSED], aborting", ex.getMessage());
|
||||
verifyAll();
|
||||
return;
|
||||
}
|
||||
|
@ -585,7 +599,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.resume(TEST_ID);
|
||||
client.resumeAsync(TEST_ID).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -613,7 +627,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.setEndOffsets(TEST_ID, endOffsets, true);
|
||||
client.setEndOffsetsAsync(TEST_ID, endOffsets, true).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -642,7 +656,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.setEndOffsets(TEST_ID, endOffsets, true);
|
||||
client.setEndOffsetsAsync(TEST_ID, endOffsets, true).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -669,7 +683,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.stop(TEST_ID, false);
|
||||
client.stopAsync(TEST_ID, false).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -695,7 +709,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
);
|
||||
replayAll();
|
||||
|
||||
client.stop(TEST_ID, true);
|
||||
client.stopAsync(TEST_ID, true).get();
|
||||
verifyAll();
|
||||
|
||||
Request request = captured.getValue();
|
||||
|
@ -1065,7 +1079,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
return Futures.immediateFuture(Either.error(responseHolder));
|
||||
}
|
||||
|
||||
private class TestableKinesisIndexTaskClient extends KinesisIndexTaskClient
|
||||
private class TestableKinesisIndexTaskClient extends SeekableStreamIndexTaskClientSyncImpl<String, String>
|
||||
{
|
||||
TestableKinesisIndexTaskClient(
|
||||
HttpClient httpClient,
|
||||
|
@ -1089,6 +1103,19 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport
|
|||
@Override
|
||||
protected void checkConnection(String host, int port)
|
||||
{
|
||||
// Do nothing.
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<String> getPartitionType()
|
||||
{
|
||||
return String.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<String> getSequenceType()
|
||||
{
|
||||
return String.class;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -299,6 +299,7 @@ public class KinesisIndexTaskTuningConfigTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
1000,
|
||||
500,
|
||||
500,
|
||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.druid.indexing.common.task.RealtimeIndexTask;
|
|||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata;
|
||||
import org.apache.druid.indexing.kinesis.KinesisIndexTask;
|
||||
import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient;
|
||||
import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory;
|
||||
import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig;
|
||||
import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig;
|
||||
|
@ -60,12 +59,14 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
|
|||
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig;
|
||||
import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
|
||||
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.LagBasedAutoScalerConfig;
|
||||
|
@ -92,8 +93,9 @@ import org.easymock.CaptureType;
|
|||
import org.easymock.EasyMock;
|
||||
import org.easymock.EasyMockSupport;
|
||||
import org.easymock.IAnswer;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.hamcrest.MatcherAssert;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
|
@ -145,7 +147,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
private TaskMaster taskMaster;
|
||||
private TaskRunner taskRunner;
|
||||
private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
|
||||
private KinesisIndexTaskClient taskClient;
|
||||
private SeekableStreamIndexTaskClient<String, String> taskClient;
|
||||
private TaskQueue taskQueue;
|
||||
private RowIngestionMetersFactory rowIngestionMetersFactory;
|
||||
private ExceptionCapturingServiceEmitter serviceEmitter;
|
||||
|
@ -169,7 +171,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
taskMaster = createMock(TaskMaster.class);
|
||||
taskRunner = createMock(TaskRunner.class);
|
||||
indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class);
|
||||
taskClient = createMock(KinesisIndexTaskClient.class);
|
||||
taskClient = createMock(SeekableStreamIndexTaskClient.class);
|
||||
EasyMock.expect(taskClient.resumeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(true)).anyTimes();
|
||||
taskQueue = createMock(TaskQueue.class);
|
||||
supervisorRecordSupplier = createMock(KinesisRecordSupplier.class);
|
||||
|
@ -191,6 +193,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
null,
|
||||
null,
|
||||
numThreads,
|
||||
null,
|
||||
TEST_CHAT_THREADS,
|
||||
TEST_CHAT_RETRIES,
|
||||
TEST_HTTP_TIMEOUT,
|
||||
|
@ -464,7 +467,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
null,
|
||||
false
|
||||
);
|
||||
KinesisIndexTaskClientFactory clientFactory = new KinesisIndexTaskClientFactory(null, OBJECT_MAPPER);
|
||||
KinesisIndexTaskClientFactory clientFactory = new KinesisIndexTaskClientFactory(null, null, OBJECT_MAPPER);
|
||||
KinesisSupervisor supervisor = new KinesisSupervisor(
|
||||
taskStorage,
|
||||
taskMaster,
|
||||
|
@ -1669,12 +1672,12 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
SHARD_ID0,
|
||||
"1"
|
||||
)));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1")).andReturn(Futures.immediateFuture(ImmutableMap.of(
|
||||
SHARD_ID1,
|
||||
"2",
|
||||
SHARD_ID0,
|
||||
"1"
|
||||
));
|
||||
)));
|
||||
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true);
|
||||
|
||||
TreeMap<Integer, Map<String, String>> checkpoints = new TreeMap<>();
|
||||
|
@ -1684,8 +1687,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
SHARD_ID0,
|
||||
"0"
|
||||
));
|
||||
EasyMock.expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean()))
|
||||
.andReturn(checkpoints)
|
||||
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean()))
|
||||
.andReturn(Futures.immediateFuture(checkpoints))
|
||||
.anyTimes();
|
||||
|
||||
taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
|
||||
|
@ -1830,12 +1833,12 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
SHARD_ID0,
|
||||
"1"
|
||||
)));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1")).andReturn(Futures.immediateFuture(ImmutableMap.of(
|
||||
SHARD_ID1,
|
||||
"2",
|
||||
SHARD_ID0,
|
||||
"1"
|
||||
));
|
||||
)));
|
||||
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true);
|
||||
|
||||
taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
|
||||
|
@ -2008,12 +2011,12 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
SHARD_ID0,
|
||||
"1"
|
||||
)));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1")).andReturn(Futures.immediateFuture(ImmutableMap.of(
|
||||
SHARD_ID1,
|
||||
"2",
|
||||
SHARD_ID0,
|
||||
"1"
|
||||
));
|
||||
)));
|
||||
EasyMock.expect(taskClient.getCurrentOffsetsAsync("id2", false))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of(
|
||||
SHARD_ID1,
|
||||
|
@ -2548,12 +2551,12 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.READING));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1")).andReturn(Futures.immediateFuture(ImmutableMap.of(
|
||||
SHARD_ID1,
|
||||
"3",
|
||||
SHARD_ID0,
|
||||
"1"
|
||||
));
|
||||
)));
|
||||
|
||||
// getCheckpoints will not be called for id1 as it is in publishing state
|
||||
TreeMap<Integer, Map<String, String>> checkpoints = new TreeMap<>();
|
||||
|
@ -2964,7 +2967,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.READING));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1")).andReturn(Futures.immediateFuture(ImmutableMap.of(
|
||||
SHARD_ID1,
|
||||
|
||||
"3",
|
||||
|
@ -2972,7 +2975,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
SHARD_ID0,
|
||||
|
||||
"1"
|
||||
));
|
||||
)));
|
||||
|
||||
TreeMap<Integer, Map<String, String>> checkpoints = new TreeMap<>();
|
||||
checkpoints.put(0, ImmutableMap.of(
|
||||
|
@ -3440,8 +3443,11 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
Assert.assertTrue(serviceEmitter.getStackTrace()
|
||||
.startsWith("org.apache.druid.java.util.common.ISE: Cannot find"));
|
||||
MatcherAssert.assertThat(
|
||||
serviceEmitter.getStackTrace(),
|
||||
CoreMatchers.startsWith("org.apache.druid.java.util.common.ISE: Cannot find taskGroup")
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"Cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]",
|
||||
serviceEmitter.getExceptionMessage()
|
||||
|
@ -3598,12 +3604,12 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.READING));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
|
||||
EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(
|
||||
EasyMock.expect(taskClient.getEndOffsetsAsync("id1")).andReturn(Futures.immediateFuture(ImmutableMap.of(
|
||||
SHARD_ID1,
|
||||
"3",
|
||||
SHARD_ID0,
|
||||
"1"
|
||||
));
|
||||
)));
|
||||
|
||||
// getCheckpoints will not be called for id1 as it is in publishing state
|
||||
TreeMap<Integer, Map<String, String>> checkpoints = new TreeMap<>();
|
||||
|
@ -3952,6 +3958,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
null,
|
||||
null,
|
||||
numThreads,
|
||||
null,
|
||||
TEST_CHAT_THREADS,
|
||||
TEST_CHAT_RETRIES,
|
||||
TEST_HTTP_TIMEOUT,
|
||||
|
@ -4971,22 +4978,22 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
);
|
||||
|
||||
KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory(
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public KinesisIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
public SeekableStreamIndexTaskClient<String, String> build(
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(TEST_CHAT_THREADS, numThreads);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout);
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, numRetries);
|
||||
Assert.assertEquals(replicas * taskCount, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
}
|
||||
};
|
||||
|
@ -5008,6 +5015,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
null,
|
||||
null,
|
||||
numThreads,
|
||||
null,
|
||||
TEST_CHAT_THREADS,
|
||||
TEST_CHAT_RETRIES,
|
||||
TEST_HTTP_TIMEOUT,
|
||||
|
@ -5114,22 +5122,25 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
);
|
||||
|
||||
KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory(
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public KinesisIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
public SeekableStreamIndexTaskClient<String, String> build(
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(TEST_CHAT_THREADS, numThreads);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout);
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, numRetries);
|
||||
Assert.assertEquals(
|
||||
replicas * (autoScalerConfig != null ? autoScalerConfig.getTaskCountMax() : taskCount),
|
||||
maxNumTasks
|
||||
);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
}
|
||||
};
|
||||
|
@ -5202,22 +5213,22 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
);
|
||||
|
||||
KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory(
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public KinesisIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
public SeekableStreamIndexTaskClient<String, String> build(
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(TEST_CHAT_THREADS, numThreads);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout);
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, numRetries);
|
||||
Assert.assertEquals(replicas * taskCount, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
}
|
||||
};
|
||||
|
@ -5292,22 +5303,22 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
);
|
||||
|
||||
KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory(
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public KinesisIndexTaskClient build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
public SeekableStreamIndexTaskClient<String, String> build(
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(TEST_CHAT_THREADS, numThreads);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout);
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, numRetries);
|
||||
Assert.assertEquals(replicas * taskCount, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -18,7 +18,8 @@
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>druid-indexing-service</artifactId>
|
||||
|
@ -382,6 +383,18 @@
|
|||
</resources>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
<version>${jacoco.version}</version>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<!-- Tested in integration tests, but we lack unit tests.
|
||||
(The newer async implementation does have unit tests.) -->
|
||||
<exclude>org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientSyncImpl.class</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
|
|
|
@ -87,11 +87,11 @@ public abstract class IndexTaskClient implements AutoCloseable
|
|||
}
|
||||
}
|
||||
|
||||
public static final int MIN_RETRY_WAIT_SECONDS = 2;
|
||||
public static final int MAX_RETRY_WAIT_SECONDS = 10;
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(IndexTaskClient.class);
|
||||
private static final String BASE_PATH = "/druid/worker/v1/chat";
|
||||
private static final int MIN_RETRY_WAIT_SECONDS = 2;
|
||||
private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5;
|
||||
|
||||
private final HttpClient httpClient;
|
||||
|
@ -115,7 +115,7 @@ public abstract class IndexTaskClient implements AutoCloseable
|
|||
this.objectMapper = objectMapper;
|
||||
this.taskInfoProvider = taskInfoProvider;
|
||||
this.httpTimeout = httpTimeout;
|
||||
this.retryPolicyFactory = initializeRetryPolicyFactory(numRetries);
|
||||
this.retryPolicyFactory = makeRetryPolicyFactory(numRetries);
|
||||
this.executorService = MoreExecutors.listeningDecorator(
|
||||
Execs.multiThreaded(
|
||||
numThreads,
|
||||
|
@ -127,7 +127,7 @@ public abstract class IndexTaskClient implements AutoCloseable
|
|||
);
|
||||
}
|
||||
|
||||
private static RetryPolicyFactory initializeRetryPolicyFactory(long numRetries)
|
||||
public static RetryPolicyFactory makeRetryPolicyFactory(long numRetries)
|
||||
{
|
||||
// Retries [numRetries] times before giving up; this should be set long enough to handle any temporary
|
||||
// unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in
|
||||
|
|
|
@ -20,409 +20,144 @@
|
|||
package org.apache.druid.indexing.seekablestream;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.indexing.common.IndexTaskClient;
|
||||
import org.apache.druid.indexing.common.RetryPolicy;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
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.HttpClient;
|
||||
import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
|
||||
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 javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
public abstract class SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType> extends IndexTaskClient
|
||||
public interface SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType>
|
||||
{
|
||||
private static final TypeReference<List<ParseExceptionReport>> TYPE_REFERENCE_LIST_PARSE_EXCEPTION_REPORT =
|
||||
new TypeReference<List<ParseExceptionReport>>()
|
||||
{
|
||||
};
|
||||
TypeReference<List<ParseExceptionReport>> TYPE_REFERENCE_LIST_PARSE_EXCEPTION_REPORT =
|
||||
new TypeReference<List<ParseExceptionReport>>() {};
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClient.class);
|
||||
/**
|
||||
* Retrieve current task checkpoints.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#getCheckpointsHTTP}.
|
||||
*
|
||||
* @param id task id
|
||||
* @param retry whether to retry on failure
|
||||
*/
|
||||
ListenableFuture<TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>>> getCheckpointsAsync(
|
||||
String id,
|
||||
boolean retry
|
||||
);
|
||||
|
||||
public SeekableStreamIndexTaskClient(
|
||||
HttpClient httpClient,
|
||||
ObjectMapper jsonMapper,
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
)
|
||||
{
|
||||
super(httpClient, jsonMapper, taskInfoProvider, httpTimeout, dataSource, numThreads, numRetries);
|
||||
}
|
||||
/**
|
||||
* Stop a task. Retries on failure. Returns true if the task was stopped, or false if it was not stopped, perhaps
|
||||
* due to an error.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#stop}.
|
||||
*
|
||||
* @param id task id
|
||||
* @param publish whether to publish already-processed data before stopping
|
||||
*/
|
||||
ListenableFuture<Boolean> stopAsync(String id, boolean publish);
|
||||
|
||||
public boolean stop(final String id, final boolean publish)
|
||||
{
|
||||
log.debug("Stop task[%s] publish[%s]", id, publish);
|
||||
/**
|
||||
* Resume a task after a call to {@link #pauseAsync}. Retries on failure. Returns true if the task was
|
||||
* resumed, or false if it was not resumed, perhaps due to an error.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#resumeHTTP}.
|
||||
*
|
||||
* @param id task id
|
||||
*/
|
||||
ListenableFuture<Boolean> resumeAsync(String id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.POST,
|
||||
"stop",
|
||||
publish ? "publish=true" : null,
|
||||
true
|
||||
);
|
||||
return isSuccess(response);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return false;
|
||||
}
|
||||
catch (TaskNotRunnableException e) {
|
||||
log.info("Task [%s] couldn't be stopped because it is no longer running", id);
|
||||
return true;
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Exception while stopping task [%s]", id);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Get the time a task actually started executing. May be later than the start time reported by the task runner,
|
||||
* since there is some delay between a task being scheduled and it actually starting to execute.
|
||||
*
|
||||
* Returns a future that resolves to null if the task has not yet started up.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#getStartTime}.
|
||||
*
|
||||
* @param id task id
|
||||
*/
|
||||
ListenableFuture<DateTime> getStartTimeAsync(String id);
|
||||
|
||||
public boolean resume(final String id)
|
||||
{
|
||||
log.debug("Resume task[%s]", id);
|
||||
/**
|
||||
* Pause a task.
|
||||
*
|
||||
* Calls {@link SeekableStreamIndexTaskRunner#pauseHTTP} task-side to do the initial pause, then uses
|
||||
* {@link SeekableStreamIndexTaskRunner#getStatusHTTP} in a loop to wait for the task to pause, then uses
|
||||
* {@link SeekableStreamIndexTaskRunner#getCurrentOffsets} to retrieve the post-pause offsets.
|
||||
*
|
||||
* @param id task id
|
||||
*/
|
||||
ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> pauseAsync(String id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.POST, "resume", null, true);
|
||||
return isSuccess(response);
|
||||
}
|
||||
catch (NoTaskLocationException | IOException e) {
|
||||
log.warn(e, "Exception while stopping task [%s]", id);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Set end offsets for a task. Retries on failure.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#setEndOffsetsHTTP}.
|
||||
*
|
||||
* @param id task id
|
||||
* @param endOffsets the end offsets
|
||||
* @param finalize whether these are the final offsets for a task (true) or an incremental checkpoint (false)
|
||||
*/
|
||||
ListenableFuture<Boolean> setEndOffsetsAsync(
|
||||
String id,
|
||||
Map<PartitionIdType, SequenceOffsetType> endOffsets,
|
||||
boolean finalize
|
||||
);
|
||||
|
||||
public Map<PartitionIdType, SequenceOffsetType> pause(final String id)
|
||||
{
|
||||
log.debug("Pause task[%s]", id);
|
||||
/**
|
||||
* Retrieve current offsets for a task.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#getCurrentOffsets}.
|
||||
*
|
||||
* @param id task id
|
||||
* @param retry whether to retry on failure
|
||||
*/
|
||||
ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getCurrentOffsetsAsync(String id, boolean retry);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.POST,
|
||||
"pause",
|
||||
null,
|
||||
true
|
||||
);
|
||||
/**
|
||||
* Retrieve ending offsets for a task. Retries on failure.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#getEndOffsetsHTTP}.
|
||||
*
|
||||
* @param id task id
|
||||
*/
|
||||
ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getEndOffsetsAsync(String id);
|
||||
|
||||
final HttpResponseStatus responseStatus = response.getStatus();
|
||||
final String responseContent = response.getContent();
|
||||
/**
|
||||
* Get processing statistics for a task. Retries on failure.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#getRowStats}.
|
||||
*
|
||||
* @param id task id
|
||||
*/
|
||||
ListenableFuture<Map<String, Object>> getMovingAveragesAsync(String id);
|
||||
|
||||
if (responseStatus.equals(HttpResponseStatus.OK)) {
|
||||
log.info("Task [%s] paused successfully", id);
|
||||
return deserializeMap(responseContent, Map.class, getPartitionType(), getSequenceType());
|
||||
} else if (responseStatus.equals(HttpResponseStatus.ACCEPTED)) {
|
||||
// The task received the pause request, but its status hasn't been changed yet.
|
||||
final RetryPolicy retryPolicy = newRetryPolicy();
|
||||
while (true) {
|
||||
final SeekableStreamIndexTaskRunner.Status status = getStatus(id);
|
||||
if (status == SeekableStreamIndexTaskRunner.Status.PAUSED) {
|
||||
return getCurrentOffsets(id, true);
|
||||
}
|
||||
/**
|
||||
* Get parse errors for a task. Retries on failure.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#getUnparseableEvents}.
|
||||
*
|
||||
* @param id task id
|
||||
*/
|
||||
ListenableFuture<List<ParseExceptionReport>> getParseErrorsAsync(String id);
|
||||
|
||||
final Duration delay = retryPolicy.getAndIncrementRetryDelay();
|
||||
if (delay == null) {
|
||||
throw new ISE(
|
||||
"Task [%s] failed to change its status from [%s] to [%s], aborting",
|
||||
id,
|
||||
status,
|
||||
SeekableStreamIndexTaskRunner.Status.PAUSED
|
||||
);
|
||||
} else {
|
||||
final long sleepTime = delay.getMillis();
|
||||
log.info(
|
||||
"Still waiting for task [%s] to change its status to [%s]; will try again in [%s]",
|
||||
id,
|
||||
SeekableStreamIndexTaskRunner.Status.PAUSED,
|
||||
new Duration(sleepTime).toString()
|
||||
);
|
||||
Thread.sleep(sleepTime);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
throw new ISE(
|
||||
"Pause request for task [%s] failed with response [%s] : [%s]",
|
||||
id,
|
||||
responseStatus,
|
||||
responseContent
|
||||
);
|
||||
}
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
log.error("Exception [%s] while pausing Task [%s]", e.getMessage(), id);
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
catch (IOException | InterruptedException e) {
|
||||
throw new RE(e, "Exception [%s] while pausing Task [%s]", e.getMessage(), id);
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Get current status for a task. Retries on failure.
|
||||
*
|
||||
* Returns {@link SeekableStreamIndexTaskRunner.Status#NOT_STARTED} if the task has not yet started.
|
||||
*
|
||||
* Task-side is {@link SeekableStreamIndexTaskRunner#getStatusHTTP}.
|
||||
*
|
||||
* @param id task id
|
||||
*/
|
||||
ListenableFuture<SeekableStreamIndexTaskRunner.Status> getStatusAsync(String id);
|
||||
|
||||
public SeekableStreamIndexTaskRunner.Status getStatus(final String id)
|
||||
{
|
||||
log.debug("GetStatus task[%s]", id);
|
||||
Class<PartitionIdType> getPartitionType();
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "status", null, true);
|
||||
return deserialize(response.getContent(), SeekableStreamIndexTaskRunner.Status.class);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return SeekableStreamIndexTaskRunner.Status.NOT_STARTED;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
Class<SequenceOffsetType> getSequenceType();
|
||||
|
||||
@Nullable
|
||||
public DateTime getStartTime(final String id)
|
||||
{
|
||||
log.debug("GetStartTime task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "time/start", null, true);
|
||||
return response.getContent() == null || response.getContent().isEmpty()
|
||||
? null
|
||||
: deserialize(response.getContent(), DateTime.class);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return null;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public Map<String, Object> getMovingAverages(final String id)
|
||||
{
|
||||
log.debug("GetMovingAverages task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.GET,
|
||||
"rowStats",
|
||||
null,
|
||||
true
|
||||
);
|
||||
if (response.getContent() == null || response.getContent().isEmpty()) {
|
||||
log.warn("Got empty response when calling getMovingAverages, id[%s]", id);
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
return deserialize(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
log.warn(e, "Got NoTaskLocationException when calling getMovingAverages, id[%s]", id);
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public List<ParseExceptionReport> getParseErrors(final String id)
|
||||
{
|
||||
log.debug("getParseErrors task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.GET,
|
||||
"unparseableEvents",
|
||||
null,
|
||||
true
|
||||
);
|
||||
|
||||
if (response.getContent() == null || response.getContent().isEmpty()) {
|
||||
log.warn("Got empty response when calling getParseErrors, id[%s]", id);
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
return deserialize(response.getContent(), TYPE_REFERENCE_LIST_PARSE_EXCEPTION_REPORT);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
log.warn(e, "Got NoTaskLocationException when calling getParseErrors, id[%s]", id);
|
||||
return Collections.emptyList();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public Map<PartitionIdType, SequenceOffsetType> getCurrentOffsets(final String id, final boolean retry)
|
||||
{
|
||||
log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.GET,
|
||||
"offsets/current",
|
||||
null,
|
||||
retry
|
||||
);
|
||||
return deserializeMap(response.getContent(), Map.class, getPartitionType(), getSequenceType());
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>> getCheckpoints(final String id, final boolean retry)
|
||||
{
|
||||
log.debug("GetCheckpoints task[%s] retry[%s]", id, retry);
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry);
|
||||
return deserializeNestedValueMap(
|
||||
response.getContent(),
|
||||
TreeMap.class,
|
||||
Integer.class,
|
||||
Map.class,
|
||||
getPartitionType(),
|
||||
getSequenceType()
|
||||
);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return new TreeMap<>();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public ListenableFuture<TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>>> getCheckpointsAsync(
|
||||
final String id,
|
||||
final boolean retry
|
||||
)
|
||||
{
|
||||
return doAsync(() -> getCheckpoints(id, retry));
|
||||
}
|
||||
|
||||
public Map<PartitionIdType, SequenceOffsetType> getEndOffsets(final String id)
|
||||
{
|
||||
log.debug("GetEndOffsets task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true);
|
||||
return deserializeMap(response.getContent(), Map.class, getPartitionType(), getSequenceType());
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean setEndOffsets(
|
||||
final String id,
|
||||
final Map<PartitionIdType, SequenceOffsetType> endOffsets,
|
||||
final boolean finalize
|
||||
) throws IOException
|
||||
{
|
||||
log.debug("SetEndOffsets task[%s] endOffsets[%s] finalize[%s]", id, endOffsets, finalize);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitJsonRequest(
|
||||
id,
|
||||
HttpMethod.POST,
|
||||
"offsets/end",
|
||||
StringUtils.format("finish=%s", finalize),
|
||||
serialize(endOffsets),
|
||||
true
|
||||
);
|
||||
return isSuccess(response);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public ListenableFuture<Boolean> stopAsync(final String id, final boolean publish)
|
||||
{
|
||||
return doAsync(() -> stop(id, publish));
|
||||
}
|
||||
|
||||
|
||||
public ListenableFuture<Boolean> resumeAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> resume(id));
|
||||
}
|
||||
|
||||
|
||||
public ListenableFuture<DateTime> getStartTimeAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getStartTime(id));
|
||||
}
|
||||
|
||||
|
||||
public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> pauseAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> pause(id));
|
||||
}
|
||||
|
||||
public ListenableFuture<Boolean> setEndOffsetsAsync(
|
||||
final String id,
|
||||
final Map<PartitionIdType, SequenceOffsetType> endOffsets,
|
||||
final boolean finalize
|
||||
)
|
||||
{
|
||||
return doAsync(() -> setEndOffsets(id, endOffsets, finalize));
|
||||
}
|
||||
|
||||
public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getCurrentOffsetsAsync(
|
||||
final String id,
|
||||
final boolean retry
|
||||
)
|
||||
{
|
||||
return doAsync(() -> getCurrentOffsets(id, retry));
|
||||
}
|
||||
|
||||
public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getEndOffsetsAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getEndOffsets(id));
|
||||
}
|
||||
|
||||
|
||||
public ListenableFuture<Map<String, Object>> getMovingAveragesAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getMovingAverages(id));
|
||||
}
|
||||
|
||||
public ListenableFuture<List<ParseExceptionReport>> getParseErrorsAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getParseErrors(id));
|
||||
}
|
||||
|
||||
public ListenableFuture<SeekableStreamIndexTaskRunner.Status> getStatusAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getStatus(id));
|
||||
}
|
||||
|
||||
protected abstract Class<PartitionIdType> getPartitionType();
|
||||
|
||||
protected abstract Class<SequenceOffsetType> getSequenceType();
|
||||
void close();
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,655 @@
|
|||
/*
|
||||
* 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.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.IndexTaskClient;
|
||||
import org.apache.druid.indexing.common.RetryPolicy;
|
||||
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.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 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}.
|
||||
*
|
||||
* Used when {@link org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig#getChatAsync()}
|
||||
* is true.
|
||||
*/
|
||||
public abstract class SeekableStreamIndexTaskClientAsyncImpl<PartitionIdType, SequenceOffsetType>
|
||||
implements SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClientAsyncImpl.class);
|
||||
|
||||
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 -> Either.value(false))
|
||||
.onNotAvailable(e -> Either.value(false))
|
||||
.onClosed(e -> {
|
||||
log.debug("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> 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, IndexTaskClient.makeRetryPolicyFactory(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);
|
||||
}
|
||||
}
|
||||
),
|
||||
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());
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
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(IndexTaskClient.MIN_RETRY_WAIT_SECONDS * 1000)
|
||||
.maxWaitMillis(IndexTaskClient.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.
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,46 +20,125 @@
|
|||
package org.apache.druid.indexing.seekablestream;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
import org.joda.time.Duration;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
|
||||
public abstract class SeekableStreamIndexTaskClientFactory<T extends SeekableStreamIndexTaskClient>
|
||||
implements IndexTaskClientFactory<T>
|
||||
public abstract class SeekableStreamIndexTaskClientFactory<PartitionIdType, SequenceOffsetType>
|
||||
{
|
||||
private HttpClient httpClient;
|
||||
private ObjectMapper mapper;
|
||||
private static final Logger log = new Logger(SeekableStreamIndexTaskClientFactory.class);
|
||||
|
||||
@Inject
|
||||
public SeekableStreamIndexTaskClientFactory(
|
||||
@EscalatedGlobal HttpClient httpClient,
|
||||
@Json ObjectMapper mapper
|
||||
private final ServiceClientFactory serviceClientFactory;
|
||||
private final HttpClient httpClient;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
protected SeekableStreamIndexTaskClientFactory(
|
||||
final ServiceClientFactory serviceClientFactory,
|
||||
final HttpClient httpClient,
|
||||
final ObjectMapper jsonMapper
|
||||
)
|
||||
{
|
||||
this.serviceClientFactory = serviceClientFactory;
|
||||
this.httpClient = httpClient;
|
||||
this.mapper = mapper;
|
||||
this.jsonMapper = jsonMapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public abstract T build(
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
);
|
||||
|
||||
protected HttpClient getHttpClient()
|
||||
public SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType> build(
|
||||
final String dataSource,
|
||||
final TaskInfoProvider taskInfoProvider,
|
||||
final int maxNumTasks,
|
||||
final SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
return httpClient;
|
||||
if (tuningConfig.getChatAsync()) {
|
||||
return buildAsync(dataSource, taskInfoProvider, tuningConfig);
|
||||
} else {
|
||||
return buildSync(dataSource, taskInfoProvider, maxNumTasks, tuningConfig);
|
||||
}
|
||||
}
|
||||
|
||||
protected ObjectMapper getMapper()
|
||||
SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType> buildAsync(
|
||||
final String dataSource,
|
||||
final TaskInfoProvider taskInfoProvider,
|
||||
final SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
return mapper;
|
||||
log.info(
|
||||
"Created async task client for dataSource[%s] httpTimeout[%s] chatRetries[%d]",
|
||||
dataSource,
|
||||
tuningConfig.getHttpTimeout(),
|
||||
tuningConfig.getChatRetries()
|
||||
);
|
||||
|
||||
return new SeekableStreamIndexTaskClientAsyncImpl<PartitionIdType, SequenceOffsetType>(
|
||||
dataSource,
|
||||
serviceClientFactory,
|
||||
taskInfoProvider,
|
||||
jsonMapper,
|
||||
tuningConfig.getHttpTimeout(),
|
||||
tuningConfig.getChatRetries()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public Class<PartitionIdType> getPartitionType()
|
||||
{
|
||||
return SeekableStreamIndexTaskClientFactory.this.getPartitionType();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<SequenceOffsetType> getSequenceType()
|
||||
{
|
||||
return SeekableStreamIndexTaskClientFactory.this.getSequenceType();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType> buildSync(
|
||||
final String dataSource,
|
||||
final TaskInfoProvider taskInfoProvider,
|
||||
final int maxNumTasks,
|
||||
final SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
final int chatThreads = (tuningConfig.getChatThreads() != null
|
||||
? tuningConfig.getChatThreads()
|
||||
: Math.min(10, maxNumTasks));
|
||||
|
||||
log.info(
|
||||
"Created taskClient for dataSource[%s] chatThreads[%d] httpTimeout[%s] chatRetries[%d]",
|
||||
dataSource,
|
||||
chatThreads,
|
||||
tuningConfig.getHttpTimeout(),
|
||||
tuningConfig.getChatRetries()
|
||||
);
|
||||
|
||||
return new SeekableStreamIndexTaskClientSyncImpl<PartitionIdType, SequenceOffsetType>(
|
||||
httpClient,
|
||||
jsonMapper,
|
||||
taskInfoProvider,
|
||||
dataSource,
|
||||
chatThreads,
|
||||
tuningConfig.getHttpTimeout(),
|
||||
tuningConfig.getChatRetries()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public Class<PartitionIdType> getPartitionType()
|
||||
{
|
||||
return SeekableStreamIndexTaskClientFactory.this.getPartitionType();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<SequenceOffsetType> getSequenceType()
|
||||
{
|
||||
return SeekableStreamIndexTaskClientFactory.this.getSequenceType();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
protected abstract Class<PartitionIdType> getPartitionType();
|
||||
|
||||
protected abstract Class<SequenceOffsetType> getSequenceType();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,433 @@
|
|||
/*
|
||||
* 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.google.common.collect.ImmutableMap;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.indexing.common.IndexTaskClient;
|
||||
import org.apache.druid.indexing.common.RetryPolicy;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
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.HttpClient;
|
||||
import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
|
||||
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 javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
/**
|
||||
* Implementation of {@link SeekableStreamIndexTaskClient} based on {@link IndexTaskClient}.
|
||||
*
|
||||
* Communication is inherently synchronous. Async operations are enabled by scheduling blocking operations on
|
||||
* a thread pool.
|
||||
*
|
||||
* Used when {@link org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig#getChatAsync()}
|
||||
* is false.
|
||||
*/
|
||||
public abstract class SeekableStreamIndexTaskClientSyncImpl<PartitionIdType, SequenceOffsetType>
|
||||
extends IndexTaskClient
|
||||
implements SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClient.class);
|
||||
|
||||
public SeekableStreamIndexTaskClientSyncImpl(
|
||||
HttpClient httpClient,
|
||||
ObjectMapper jsonMapper,
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
String dataSource,
|
||||
int numThreads,
|
||||
Duration httpTimeout,
|
||||
long numRetries
|
||||
)
|
||||
{
|
||||
super(httpClient, jsonMapper, taskInfoProvider, httpTimeout, dataSource, numThreads, numRetries);
|
||||
}
|
||||
|
||||
private boolean stop(final String id, final boolean publish)
|
||||
{
|
||||
log.debug("Stop task[%s] publish[%s]", id, publish);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.POST,
|
||||
"stop",
|
||||
publish ? "publish=true" : null,
|
||||
true
|
||||
);
|
||||
return isSuccess(response);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return false;
|
||||
}
|
||||
catch (TaskNotRunnableException e) {
|
||||
log.info("Task [%s] couldn't be stopped because it is no longer running", id);
|
||||
return true;
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Exception while stopping task [%s]", id);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private boolean resume(final String id)
|
||||
{
|
||||
log.debug("Resume task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.POST, "resume", null, true);
|
||||
return isSuccess(response);
|
||||
}
|
||||
catch (NoTaskLocationException | IOException e) {
|
||||
log.warn(e, "Exception while stopping task [%s]", id);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private Map<PartitionIdType, SequenceOffsetType> pause(final String id)
|
||||
{
|
||||
log.debug("Pause task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.POST,
|
||||
"pause",
|
||||
null,
|
||||
true
|
||||
);
|
||||
|
||||
final HttpResponseStatus responseStatus = response.getStatus();
|
||||
final String responseContent = response.getContent();
|
||||
|
||||
if (responseStatus.equals(HttpResponseStatus.OK)) {
|
||||
log.info("Task [%s] paused successfully", id);
|
||||
return deserializeMap(responseContent, Map.class, getPartitionType(), getSequenceType());
|
||||
} else if (responseStatus.equals(HttpResponseStatus.ACCEPTED)) {
|
||||
// The task received the pause request, but its status hasn't been changed yet.
|
||||
final RetryPolicy retryPolicy = newRetryPolicy();
|
||||
while (true) {
|
||||
final SeekableStreamIndexTaskRunner.Status status = getStatus(id);
|
||||
if (status == SeekableStreamIndexTaskRunner.Status.PAUSED) {
|
||||
return getCurrentOffsets(id, true);
|
||||
}
|
||||
|
||||
final Duration delay = retryPolicy.getAndIncrementRetryDelay();
|
||||
if (delay == null) {
|
||||
throw new ISE(
|
||||
"Task [%s] failed to change its status from [%s] to [%s], aborting",
|
||||
id,
|
||||
status,
|
||||
SeekableStreamIndexTaskRunner.Status.PAUSED
|
||||
);
|
||||
} else {
|
||||
final long sleepTime = delay.getMillis();
|
||||
log.info(
|
||||
"Still waiting for task [%s] to change its status to [%s]; will try again in [%s]",
|
||||
id,
|
||||
SeekableStreamIndexTaskRunner.Status.PAUSED,
|
||||
new Duration(sleepTime).toString()
|
||||
);
|
||||
Thread.sleep(sleepTime);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
throw new ISE(
|
||||
"Pause request for task [%s] failed with response [%s] : [%s]",
|
||||
id,
|
||||
responseStatus,
|
||||
responseContent
|
||||
);
|
||||
}
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
log.error("Exception [%s] while pausing Task [%s]", e.getMessage(), id);
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
catch (IOException | InterruptedException e) {
|
||||
throw new RE(e, "Exception [%s] while pausing Task [%s]", e.getMessage(), id);
|
||||
}
|
||||
}
|
||||
|
||||
private SeekableStreamIndexTaskRunner.Status getStatus(final String id)
|
||||
{
|
||||
log.debug("GetStatus task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "status", null, true);
|
||||
return deserialize(response.getContent(), SeekableStreamIndexTaskRunner.Status.class);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return SeekableStreamIndexTaskRunner.Status.NOT_STARTED;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private DateTime getStartTime(final String id)
|
||||
{
|
||||
log.debug("GetStartTime task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "time/start", null, true);
|
||||
return response.getContent() == null || response.getContent().isEmpty()
|
||||
? null
|
||||
: deserialize(response.getContent(), DateTime.class);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return null;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, Object> getMovingAverages(final String id)
|
||||
{
|
||||
log.debug("GetMovingAverages task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.GET,
|
||||
"rowStats",
|
||||
null,
|
||||
true
|
||||
);
|
||||
if (response.getContent() == null || response.getContent().isEmpty()) {
|
||||
log.warn("Got empty response when calling getMovingAverages, id[%s]", id);
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
return deserialize(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
log.warn(e, "Got NoTaskLocationException when calling getMovingAverages, id[%s]", id);
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private List<ParseExceptionReport> getParseErrors(final String id)
|
||||
{
|
||||
log.debug("getParseErrors task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.GET,
|
||||
"unparseableEvents",
|
||||
null,
|
||||
true
|
||||
);
|
||||
|
||||
if (response.getContent() == null || response.getContent().isEmpty()) {
|
||||
log.warn("Got empty response when calling getParseErrors, id[%s]", id);
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
return deserialize(response.getContent(), TYPE_REFERENCE_LIST_PARSE_EXCEPTION_REPORT);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
log.warn(e, "Got NoTaskLocationException when calling getParseErrors, id[%s]", id);
|
||||
return Collections.emptyList();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private Map<PartitionIdType, SequenceOffsetType> getCurrentOffsets(final String id, final boolean retry)
|
||||
{
|
||||
log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(
|
||||
id,
|
||||
HttpMethod.GET,
|
||||
"offsets/current",
|
||||
null,
|
||||
retry
|
||||
);
|
||||
return deserializeMap(response.getContent(), Map.class, getPartitionType(), getSequenceType());
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>> getCheckpoints(final String id, final boolean retry)
|
||||
{
|
||||
log.debug("GetCheckpoints task[%s] retry[%s]", id, retry);
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry);
|
||||
return deserializeNestedValueMap(
|
||||
response.getContent(),
|
||||
TreeMap.class,
|
||||
Integer.class,
|
||||
Map.class,
|
||||
getPartitionType(),
|
||||
getSequenceType()
|
||||
);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return new TreeMap<>();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>>> getCheckpointsAsync(
|
||||
final String id,
|
||||
final boolean retry
|
||||
)
|
||||
{
|
||||
return doAsync(() -> getCheckpoints(id, retry));
|
||||
}
|
||||
|
||||
private Map<PartitionIdType, SequenceOffsetType> getEndOffsets(final String id)
|
||||
{
|
||||
log.debug("GetEndOffsets task[%s]", id);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true);
|
||||
return deserializeMap(response.getContent(), Map.class, getPartitionType(), getSequenceType());
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean setEndOffsets(
|
||||
final String id,
|
||||
final Map<PartitionIdType, SequenceOffsetType> endOffsets,
|
||||
final boolean finalize
|
||||
) throws IOException
|
||||
{
|
||||
log.debug("SetEndOffsets task[%s] endOffsets[%s] finalize[%s]", id, endOffsets, finalize);
|
||||
|
||||
try {
|
||||
final StringFullResponseHolder response = submitJsonRequest(
|
||||
id,
|
||||
HttpMethod.POST,
|
||||
"offsets/end",
|
||||
StringUtils.format("finish=%s", finalize),
|
||||
serialize(endOffsets),
|
||||
true
|
||||
);
|
||||
return isSuccess(response);
|
||||
}
|
||||
catch (NoTaskLocationException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Boolean> stopAsync(final String id, final boolean publish)
|
||||
{
|
||||
return doAsync(() -> stop(id, publish));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Boolean> resumeAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> resume(id));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<DateTime> getStartTimeAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getStartTime(id));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> pauseAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> pause(id));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Boolean> setEndOffsetsAsync(
|
||||
final String id,
|
||||
final Map<PartitionIdType, SequenceOffsetType> endOffsets,
|
||||
final boolean finalize
|
||||
)
|
||||
{
|
||||
return doAsync(() -> setEndOffsets(id, endOffsets, finalize));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getCurrentOffsetsAsync(
|
||||
final String id,
|
||||
final boolean retry
|
||||
)
|
||||
{
|
||||
return doAsync(() -> getCurrentOffsets(id, retry));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getEndOffsetsAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getEndOffsets(id));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Map<String, Object>> getMovingAveragesAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getMovingAverages(id));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<List<ParseExceptionReport>> getParseErrorsAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getParseErrors(id));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<SeekableStreamIndexTaskRunner.Status> getStatusAsync(final String id)
|
||||
{
|
||||
return doAsync(() -> getStatus(id));
|
||||
}
|
||||
}
|
|
@ -40,6 +40,7 @@ import com.google.common.util.concurrent.MoreExecutors;
|
|||
import it.unimi.dsi.fastutil.ints.Int2ObjectLinkedOpenHashMap;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
|
||||
import org.apache.commons.codec.digest.DigestUtils;
|
||||
import org.apache.druid.common.guava.FutureUtils;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.indexer.TaskLocation;
|
||||
import org.apache.druid.indexer.TaskState;
|
||||
|
@ -75,6 +76,7 @@ import org.apache.druid.indexing.seekablestream.common.StreamException;
|
|||
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Either;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
|
@ -758,7 +760,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
final TaskStorage taskStorage,
|
||||
final TaskMaster taskMaster,
|
||||
final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
|
||||
final SeekableStreamIndexTaskClientFactory<? extends SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType>> taskClientFactory,
|
||||
final SeekableStreamIndexTaskClientFactory<PartitionIdType, SequenceOffsetType> taskClientFactory,
|
||||
final ObjectMapper mapper,
|
||||
final SeekableStreamSupervisorSpec spec,
|
||||
final RowIngestionMetersFactory rowIngestionMetersFactory,
|
||||
|
@ -789,7 +791,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
);
|
||||
|
||||
int workerThreads;
|
||||
int chatThreads;
|
||||
int maxNumTasks;
|
||||
if (autoScalerConfig != null && autoScalerConfig.getEnableTaskAutoScaler()) {
|
||||
log.info("Running Task autoscaler for datasource [%s]", dataSource);
|
||||
|
||||
|
@ -797,17 +799,13 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
? this.tuningConfig.getWorkerThreads()
|
||||
: Math.min(10, autoScalerConfig.getTaskCountMax()));
|
||||
|
||||
chatThreads = (this.tuningConfig.getChatThreads() != null
|
||||
? this.tuningConfig.getChatThreads()
|
||||
: Math.min(10, autoScalerConfig.getTaskCountMax() * this.ioConfig.getReplicas()));
|
||||
maxNumTasks = autoScalerConfig.getTaskCountMax() * this.ioConfig.getReplicas();
|
||||
} else {
|
||||
workerThreads = (this.tuningConfig.getWorkerThreads() != null
|
||||
? this.tuningConfig.getWorkerThreads()
|
||||
: Math.min(10, this.ioConfig.getTaskCount()));
|
||||
|
||||
chatThreads = (this.tuningConfig.getChatThreads() != null
|
||||
? this.tuningConfig.getChatThreads()
|
||||
: Math.min(10, this.ioConfig.getTaskCount() * this.ioConfig.getReplicas()));
|
||||
maxNumTasks = this.ioConfig.getTaskCount() * this.ioConfig.getReplicas();
|
||||
}
|
||||
|
||||
IdleConfig specIdleConfig = spec.getIoConfig().getIdleConfig();
|
||||
|
@ -871,23 +869,9 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
+ IndexTaskClient.MAX_RETRY_WAIT_SECONDS)
|
||||
);
|
||||
|
||||
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()
|
||||
);
|
||||
this.taskClient = taskClientFactory.build(dataSource, taskInfoProvider, maxNumTasks, this.tuningConfig);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int getActiveTaskGroupsCount()
|
||||
{
|
||||
|
@ -1052,7 +1036,14 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
Instant handleNoticeEndTime = Instant.now();
|
||||
Duration timeElapsed = Duration.between(handleNoticeStartTime, handleNoticeEndTime);
|
||||
String noticeType = notice.getType();
|
||||
log.debug("Handled notice [%s] from notices queue in [%d] ms, current notices queue size [%d] for datasource [%s]", noticeType, timeElapsed.toMillis(), getNoticesQueueSize(), dataSource);
|
||||
log.debug(
|
||||
"Handled notice [%s] from notices queue in [%d] ms, "
|
||||
+ "current notices queue size [%d] for datasource [%s]",
|
||||
noticeType,
|
||||
timeElapsed.toMillis(),
|
||||
getNoticesQueueSize(),
|
||||
dataSource
|
||||
);
|
||||
emitNoticeProcessTime(noticeType, timeElapsed.toMillis());
|
||||
}
|
||||
catch (Throwable e) {
|
||||
|
@ -1303,16 +1294,23 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
}
|
||||
|
||||
List<StatsFromTaskResult> results = Futures.successfulAsList(futures)
|
||||
.get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
List<Either<Throwable, StatsFromTaskResult>> results = coalesceAndAwait(futures);
|
||||
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());
|
||||
if (results.get(i).isValue()) {
|
||||
StatsFromTaskResult result = results.get(i).valueOrThrow();
|
||||
|
||||
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);
|
||||
log.noStackTrace().warn(
|
||||
results.get(i).error(),
|
||||
"Failed to get stats for group[%d]-task[%s]",
|
||||
groupAndTaskId.lhs,
|
||||
groupAndTaskId.rhs
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1374,15 +1372,22 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
TreeSet<ParseExceptionReport> parseErrorsTreeSet = new TreeSet<>(PARSE_EXCEPTION_REPORT_COMPARATOR);
|
||||
parseErrorsTreeSet.addAll(lastKnownParseErrors);
|
||||
|
||||
List<ErrorsFromTaskResult> results = Futures.successfulAsList(futures)
|
||||
.get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
List<Either<Throwable, ErrorsFromTaskResult>> results = coalesceAndAwait(futures);
|
||||
for (int i = 0; i < results.size(); i++) {
|
||||
ErrorsFromTaskResult result = results.get(i);
|
||||
if (result != null) {
|
||||
parseErrorsTreeSet.addAll(result.getErrors());
|
||||
if (results.get(i).isValue()) {
|
||||
ErrorsFromTaskResult result = results.get(i).valueOrThrow();
|
||||
|
||||
if (result != null) {
|
||||
parseErrorsTreeSet.addAll(result.getErrors());
|
||||
}
|
||||
} else {
|
||||
Pair<Integer, String> groupAndTaskId = groupAndTaskIds.get(i);
|
||||
log.error("Failed to get errors for group[%d]-task[%s]", groupAndTaskId.lhs, groupAndTaskId.rhs);
|
||||
log.noStackTrace().warn(
|
||||
results.get(i).error(),
|
||||
"Failed to get errors for group[%d]-task[%s]",
|
||||
groupAndTaskId.lhs,
|
||||
groupAndTaskId.rhs
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1803,12 +1808,15 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
futureTaskIds.add(taskId);
|
||||
futures.add(
|
||||
Futures.transform(
|
||||
taskClient.getStatusAsync(taskId),
|
||||
new Function<SeekableStreamIndexTaskRunner.Status, Boolean>()
|
||||
getStatusAndPossiblyEndOffsets(taskId),
|
||||
new Function<Pair<SeekableStreamIndexTaskRunner.Status, Map<PartitionIdType, SequenceOffsetType>>, Boolean>()
|
||||
{
|
||||
@Override
|
||||
public Boolean apply(SeekableStreamIndexTaskRunner.Status status)
|
||||
public Boolean apply(Pair<SeekableStreamIndexTaskRunner.Status, Map<PartitionIdType, SequenceOffsetType>> pair)
|
||||
{
|
||||
final SeekableStreamIndexTaskRunner.Status status = pair.lhs;
|
||||
final Map<PartitionIdType, SequenceOffsetType> publishingTaskEndOffsets = pair.rhs;
|
||||
|
||||
try {
|
||||
log.debug("Task [%s], status [%s]", taskId, status);
|
||||
if (status == SeekableStreamIndexTaskRunner.Status.PUBLISHING) {
|
||||
|
@ -1827,10 +1835,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
));
|
||||
|
||||
// update partitionGroups with the publishing task's sequences (if they are greater than what is
|
||||
// existing) so that the next tasks will start reading from where this task left off
|
||||
Map<PartitionIdType, SequenceOffsetType> publishingTaskEndOffsets = taskClient.getEndOffsets(
|
||||
taskId);
|
||||
|
||||
// existing) so that the next tasks will start reading from where this task left off.
|
||||
// If we received invalid endOffset values, we clear the known offset to refetch the last committed offset
|
||||
// from metadata. If any endOffset values are invalid, we treat the entire set as invalid as a safety measure.
|
||||
boolean endOffsetsAreInvalid = false;
|
||||
|
@ -1876,14 +1881,8 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
"Stopping task [%s] which does not match the expected partition allocation",
|
||||
taskId
|
||||
);
|
||||
try {
|
||||
stopTask(taskId, false)
|
||||
.get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
}
|
||||
catch (InterruptedException | ExecutionException | TimeoutException e) {
|
||||
stateManager.recordThrowableEvent(e);
|
||||
log.warn(e, "Exception while stopping task");
|
||||
}
|
||||
|
||||
// Returning false triggers a call to stopTask.
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -1894,14 +1893,8 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
"Stopping task [%s] which does not match the expected parameters and ingestion spec",
|
||||
taskId
|
||||
);
|
||||
try {
|
||||
stopTask(taskId, false)
|
||||
.get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
}
|
||||
catch (InterruptedException | ExecutionException | TimeoutException e) {
|
||||
stateManager.recordThrowableEvent(e);
|
||||
log.warn(e, "Exception while stopping task");
|
||||
}
|
||||
|
||||
// Returning false triggers a call to stopTask.
|
||||
return false;
|
||||
} else {
|
||||
final TaskGroup taskGroup = activelyReadingTaskGroups.computeIfAbsent(
|
||||
|
@ -1954,17 +1947,25 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
}
|
||||
|
||||
List<Boolean> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
List<Either<Throwable, Boolean>> results = coalesceAndAwait(futures);
|
||||
|
||||
final List<ListenableFuture<Void>> stopFutures = new ArrayList<>();
|
||||
for (int i = 0; i < results.size(); i++) {
|
||||
if (results.get(i) == null) {
|
||||
String taskId = futureTaskIds.get(i);
|
||||
String taskId = futureTaskIds.get(i);
|
||||
if (results.get(i).isError() || results.get(i).valueOrThrow() == null) {
|
||||
killTask(taskId, "Task [%s] failed to return status, killing task", taskId);
|
||||
} else if (Boolean.valueOf(false).equals(results.get(i).valueOrThrow())) {
|
||||
// "return false" above means that we want to stop the task.
|
||||
stopFutures.add(stopTask(taskId, false));
|
||||
}
|
||||
}
|
||||
log.debug("Found [%d] seekablestream indexing tasks for dataSource [%s]", taskCount, dataSource);
|
||||
|
||||
// make sure the checkpoints are consistent with each other and with the metadata store
|
||||
if (!stopFutures.isEmpty()) {
|
||||
coalesceAndAwait(stopFutures);
|
||||
}
|
||||
|
||||
// make sure the checkpoints are consistent with each other and with the metadata store
|
||||
verifyAndMergeCheckpoints(taskGroupsToVerify.values());
|
||||
|
||||
// A pause from the previous Overlord's supervisor, immediately before leader change,
|
||||
|
@ -1972,6 +1973,35 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
resumeAllActivelyReadingTasks();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a Pair of information about a task:
|
||||
*
|
||||
* Left-hand side: Status of the task from {@link SeekableStreamIndexTaskClient#getStatusAsync}.
|
||||
*
|
||||
* Right-hand side: If status is {@link SeekableStreamIndexTaskRunner.Status#PUBLISHING}, end offsets from
|
||||
* {@link SeekableStreamIndexTaskClient#getEndOffsetsAsync}. Otherwise, null.
|
||||
*
|
||||
* Used by {@link #discoverTasks()}.
|
||||
*/
|
||||
private ListenableFuture<Pair<SeekableStreamIndexTaskRunner.Status, Map<PartitionIdType, SequenceOffsetType>>> getStatusAndPossiblyEndOffsets(
|
||||
final String taskId
|
||||
)
|
||||
{
|
||||
return FutureUtils.transformAsync(
|
||||
taskClient.getStatusAsync(taskId),
|
||||
status -> {
|
||||
if (status == SeekableStreamIndexTaskRunner.Status.PUBLISHING) {
|
||||
return FutureUtils.transform(
|
||||
taskClient.getEndOffsetsAsync(taskId),
|
||||
endOffsets -> Pair.of(status, endOffsets)
|
||||
);
|
||||
} else {
|
||||
return Futures.immediateFuture(Pair.of(status, null));
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* If this is the first run, resume all tasks in the set of activelyReadingTaskGroups
|
||||
* Paused tasks will be resumed
|
||||
|
@ -2022,12 +2052,15 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
|
||||
private void verifyAndMergeCheckpoints(final Collection<TaskGroup> taskGroupsToVerify)
|
||||
{
|
||||
final List<ListenableFuture<?>> futures = new ArrayList<>();
|
||||
final List<ListenableFuture<Object>> futures = new ArrayList<>();
|
||||
for (TaskGroup taskGroup : taskGroupsToVerify) {
|
||||
futures.add(workerExec.submit(() -> verifyAndMergeCheckpoints(taskGroup)));
|
||||
//noinspection unchecked
|
||||
futures.add((ListenableFuture<Object>) workerExec.submit(() -> verifyAndMergeCheckpoints(taskGroup)));
|
||||
}
|
||||
|
||||
try {
|
||||
Futures.allAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
// Ignore return value; just await.
|
||||
coalesceAndAwait(futures);
|
||||
}
|
||||
catch (InterruptedException | ExecutionException | TimeoutException e) {
|
||||
throw new RuntimeException(e);
|
||||
|
@ -2059,26 +2092,23 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
|
||||
try {
|
||||
List<TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>>> futuresResult =
|
||||
Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
List<Either<Throwable, TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>>>> futuresResult =
|
||||
coalesceAndAwait(futures);
|
||||
|
||||
for (int i = 0; i < futuresResult.size(); i++) {
|
||||
final TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>> checkpoints = futuresResult.get(i);
|
||||
final Either<Throwable, TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>>> futureResult =
|
||||
futuresResult.get(i);
|
||||
final String taskId = taskIds.get(i);
|
||||
if (checkpoints == null) {
|
||||
try {
|
||||
// catch the exception in failed futures
|
||||
futures.get(i).get();
|
||||
}
|
||||
catch (Exception e) {
|
||||
stateManager.recordThrowableEvent(e);
|
||||
log.error(e, "Problem while getting checkpoints for task [%s], killing the task", taskId);
|
||||
killTask(taskId, "Exception[%s] while getting checkpoints", e.getClass());
|
||||
taskGroup.tasks.remove(taskId);
|
||||
}
|
||||
} else if (checkpoints.isEmpty()) {
|
||||
if (futureResult.isError()) {
|
||||
final Throwable e = new RuntimeException(futureResult.error());
|
||||
stateManager.recordThrowableEvent(e);
|
||||
log.error(e, "Problem while getting checkpoints for task [%s], killing the task", taskId);
|
||||
killTask(taskId, "Exception[%s] while getting checkpoints", e.getClass());
|
||||
taskGroup.tasks.remove(taskId);
|
||||
} else if (futureResult.valueOrThrow().isEmpty()) {
|
||||
log.warn("Ignoring task [%s], as probably it is not started running yet", taskId);
|
||||
} else {
|
||||
taskSequences.add(new Pair<>(taskId, checkpoints));
|
||||
taskSequences.add(new Pair<>(taskId, futureResult.valueOrThrow()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2864,13 +2894,12 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
}
|
||||
|
||||
List<Boolean> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
final List<Either<Throwable, Boolean>> results = coalesceAndAwait(futures);
|
||||
for (int i = 0; i < results.size(); i++) {
|
||||
// false means the task hasn't started running yet and that's okay; null means it should be running but the HTTP
|
||||
// request threw an exception so kill the task
|
||||
if (results.get(i) == null) {
|
||||
// Ignore return value; but kill tasks that failed to return anything at all.
|
||||
if (results.get(i).isError()) {
|
||||
String taskId = futureTaskIds.get(i);
|
||||
log.warn("Task [%s] failed to return start time, killing task", taskId);
|
||||
log.noStackTrace().warn(results.get(i).error(), "Task [%s] failed to return start time, killing task", taskId);
|
||||
killTask(taskId, "Task [%s] failed to return start time, killing task", taskId);
|
||||
}
|
||||
}
|
||||
|
@ -2911,14 +2940,13 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
}
|
||||
|
||||
List<Map<PartitionIdType, SequenceOffsetType>> results = Futures.successfulAsList(futures)
|
||||
.get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
List<Either<Throwable, Map<PartitionIdType, SequenceOffsetType>>> results = coalesceAndAwait(futures);
|
||||
for (int j = 0; j < results.size(); j++) {
|
||||
Integer groupId = futureGroupIds.get(j);
|
||||
TaskGroup group = activelyReadingTaskGroups.get(groupId);
|
||||
Map<PartitionIdType, SequenceOffsetType> endOffsets = results.get(j);
|
||||
|
||||
if (endOffsets != null) {
|
||||
if (results.get(j).isValue() && results.get(j).valueOrThrow() != null) {
|
||||
Map<PartitionIdType, SequenceOffsetType> endOffsets = results.get(j).valueOrThrow();
|
||||
// set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion
|
||||
group.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
|
||||
pendingCompletionTaskGroups.computeIfAbsent(groupId, k -> new CopyOnWriteArrayList<>()).add(group);
|
||||
|
@ -3019,38 +3047,22 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
|
||||
return Futures.transform(
|
||||
Futures.successfulAsList(pauseFutures),
|
||||
new Function<List<Map<PartitionIdType, SequenceOffsetType>>, Map<PartitionIdType, SequenceOffsetType>>()
|
||||
FutureUtils.coalesce(pauseFutures),
|
||||
new Function<List<Either<Throwable, Map<PartitionIdType, SequenceOffsetType>>>, Map<PartitionIdType, SequenceOffsetType>>()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Map<PartitionIdType, SequenceOffsetType> apply(List<Map<PartitionIdType, SequenceOffsetType>> input)
|
||||
public Map<PartitionIdType, SequenceOffsetType> apply(List<Either<Throwable, Map<PartitionIdType, SequenceOffsetType>>> input)
|
||||
{
|
||||
// 3) Build a map of the highest sequence read by any task in the group for each partition
|
||||
final Map<PartitionIdType, SequenceOffsetType> endOffsets = new HashMap<>();
|
||||
for (int i = 0; i < input.size(); i++) {
|
||||
final Map<PartitionIdType, SequenceOffsetType> result = input.get(i);
|
||||
final String taskId = pauseTaskIds.get(i);
|
||||
|
||||
if (result == null) {
|
||||
if (input.get(i).isError()) {
|
||||
// Get the exception
|
||||
final Throwable pauseException;
|
||||
try {
|
||||
// The below get should throw ExecutionException since result is null.
|
||||
final Map<PartitionIdType, SequenceOffsetType> pauseResult = pauseFutures.get(i).get();
|
||||
throw new ISE(
|
||||
"Pause request for task [%s] should have failed, but returned [%s]",
|
||||
taskId,
|
||||
pauseResult
|
||||
);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
catch (ExecutionException e) {
|
||||
stateManager.recordThrowableEvent(e);
|
||||
pauseException = e.getCause() == null ? e : e.getCause();
|
||||
}
|
||||
final Throwable pauseException = input.get(i).error();
|
||||
stateManager.recordThrowableEvent(pauseException);
|
||||
|
||||
killTask(
|
||||
taskId,
|
||||
|
@ -3060,11 +3072,11 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
);
|
||||
taskGroup.tasks.remove(taskId);
|
||||
|
||||
} else if (result.isEmpty()) {
|
||||
} else if (input.get(i).valueOrThrow() == null || input.get(i).valueOrThrow().isEmpty()) {
|
||||
killTask(taskId, "Task [%s] returned empty offsets after pause", taskId);
|
||||
taskGroup.tasks.remove(taskId);
|
||||
} else { // otherwise build a map of the highest sequences seen
|
||||
for (Entry<PartitionIdType, SequenceOffsetType> sequence : result.entrySet()) {
|
||||
for (Entry<PartitionIdType, SequenceOffsetType> sequence : input.get(i).valueOrThrow().entrySet()) {
|
||||
if (!endOffsets.containsKey(sequence.getKey())
|
||||
|| makeSequenceNumber(endOffsets.get(sequence.getKey())).compareTo(
|
||||
makeSequenceNumber(sequence.getValue())) < 0) {
|
||||
|
@ -3104,10 +3116,11 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
setEndOffsetFutures.add(taskClient.setEndOffsetsAsync(taskId, endOffsets, finalize));
|
||||
}
|
||||
|
||||
List<Boolean> results = Futures.successfulAsList(setEndOffsetFutures)
|
||||
.get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
List<Either<Throwable, Boolean>> results = coalesceAndAwait(setEndOffsetFutures);
|
||||
for (int i = 0; i < results.size(); i++) {
|
||||
if (results.get(i) == null || !results.get(i)) {
|
||||
if (results.get(i).isValue() && Boolean.valueOf(true).equals(results.get(i).valueOrThrow())) {
|
||||
log.info("Successfully set endOffsets for task[%s] and resumed it", setEndOffsetTaskIds.get(i));
|
||||
} else {
|
||||
String taskId = setEndOffsetTaskIds.get(i);
|
||||
killTask(
|
||||
taskId,
|
||||
|
@ -3115,8 +3128,6 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
taskId
|
||||
);
|
||||
taskGroup.tasks.remove(taskId);
|
||||
} else {
|
||||
log.info("Successfully set endOffsets for task[%s] and resumed it", setEndOffsetTaskIds.get(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -3137,7 +3148,11 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
);
|
||||
}
|
||||
|
||||
private ListenableFuture<?> stopTasksInGroup(@Nullable TaskGroup taskGroup, String stopReasonFormat, Object... args)
|
||||
private ListenableFuture<Void> stopTasksInGroup(
|
||||
@Nullable TaskGroup taskGroup,
|
||||
String stopReasonFormat,
|
||||
Object... args
|
||||
)
|
||||
{
|
||||
if (taskGroup == null) {
|
||||
return Futures.immediateFuture(null);
|
||||
|
@ -3160,13 +3175,13 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
}
|
||||
|
||||
return Futures.successfulAsList(futures);
|
||||
return FutureUtils.transform(FutureUtils.coalesce(futures), xs -> null);
|
||||
}
|
||||
|
||||
private void checkPendingCompletionTasks()
|
||||
throws ExecutionException, InterruptedException, TimeoutException
|
||||
{
|
||||
List<ListenableFuture<?>> futures = new ArrayList<>();
|
||||
List<ListenableFuture<Void>> futures = new ArrayList<>();
|
||||
|
||||
for (Entry<Integer, CopyOnWriteArrayList<TaskGroup>> pendingGroupList : pendingCompletionTaskGroups.entrySet()) {
|
||||
|
||||
|
@ -3262,15 +3277,15 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
taskGroupList.removeAll(toRemove);
|
||||
}
|
||||
|
||||
// wait for all task shutdowns to complete before returning
|
||||
Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
// Ignore return value; just await.
|
||||
coalesceAndAwait(futures);
|
||||
}
|
||||
|
||||
private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException
|
||||
{
|
||||
Map<String, Task> activeTaskMap = getActiveTaskMap();
|
||||
|
||||
List<ListenableFuture<?>> futures = new ArrayList<>();
|
||||
List<ListenableFuture<Void>> futures = new ArrayList<>();
|
||||
Iterator<Entry<Integer, TaskGroup>> iTaskGroups = activelyReadingTaskGroups.entrySet().iterator();
|
||||
while (iTaskGroups.hasNext()) {
|
||||
Entry<Integer, TaskGroup> taskGroupEntry = iTaskGroups.next();
|
||||
|
@ -3320,8 +3335,8 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.taskIds());
|
||||
}
|
||||
|
||||
// wait for all task shutdowns to complete before returning
|
||||
Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
// Ignore return value; just await.
|
||||
coalesceAndAwait(futures);
|
||||
}
|
||||
|
||||
private void checkIfStreamInactiveAndTurnSupervisorIdle()
|
||||
|
@ -3752,7 +3767,8 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
)
|
||||
).collect(Collectors.toList());
|
||||
|
||||
Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
// Ignore return value; just await.
|
||||
coalesceAndAwait(futures);
|
||||
}
|
||||
|
||||
protected abstract void updatePartitionLagFromStream();
|
||||
|
@ -4094,6 +4110,28 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Call {@link FutureUtils#coalesce} on the provided list, and wait for it up to {@link #futureTimeoutInSeconds}.
|
||||
*/
|
||||
private <T> List<Either<Throwable, T>> coalesceAndAwait(final List<ListenableFuture<T>> futures)
|
||||
throws ExecutionException, InterruptedException, TimeoutException
|
||||
{
|
||||
final ListenableFuture<List<Either<Throwable, T>>> coalesced = FutureUtils.coalesce(futures);
|
||||
|
||||
try {
|
||||
if (tuningConfig.getChatAsync()) {
|
||||
// Let the async client handle timeouts.
|
||||
return coalesced.get();
|
||||
} else {
|
||||
return coalesced.get(futureTimeoutInSeconds, TimeUnit.SECONDS);
|
||||
}
|
||||
}
|
||||
catch (InterruptedException | TimeoutException e) {
|
||||
coalesced.cancel(true);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
protected void emitNoticeProcessTime(String noticeType, long timeInMillis)
|
||||
{
|
||||
try {
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package org.apache.druid.indexing.seekablestream.supervisor;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -72,20 +70,19 @@ public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec
|
|||
private final boolean suspended;
|
||||
protected final SupervisorStateManagerConfig supervisorStateManagerConfig;
|
||||
|
||||
@JsonCreator
|
||||
public SeekableStreamSupervisorSpec(
|
||||
@JsonProperty("spec") final SeekableStreamSupervisorIngestionSpec ingestionSchema,
|
||||
@JsonProperty("context") @Nullable Map<String, Object> context,
|
||||
@JsonProperty("suspended") Boolean suspended,
|
||||
@JacksonInject TaskStorage taskStorage,
|
||||
@JacksonInject TaskMaster taskMaster,
|
||||
@JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
|
||||
@JacksonInject SeekableStreamIndexTaskClientFactory indexTaskClientFactory,
|
||||
@JacksonInject @Json ObjectMapper mapper,
|
||||
@JacksonInject ServiceEmitter emitter,
|
||||
@JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig,
|
||||
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
|
||||
@JacksonInject SupervisorStateManagerConfig supervisorStateManagerConfig
|
||||
final SeekableStreamSupervisorIngestionSpec ingestionSchema,
|
||||
@Nullable Map<String, Object> context,
|
||||
Boolean suspended,
|
||||
TaskStorage taskStorage,
|
||||
TaskMaster taskMaster,
|
||||
IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
|
||||
SeekableStreamIndexTaskClientFactory indexTaskClientFactory,
|
||||
@Json ObjectMapper mapper,
|
||||
ServiceEmitter emitter,
|
||||
DruidMonitorSchedulerConfig monitorSchedulerConfig,
|
||||
RowIngestionMetersFactory rowIngestionMetersFactory,
|
||||
SupervisorStateManagerConfig supervisorStateManagerConfig
|
||||
)
|
||||
{
|
||||
this.ingestionSchema = checkIngestionSchema(ingestionSchema);
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.joda.time.Period;
|
|||
|
||||
public interface SeekableStreamSupervisorTuningConfig
|
||||
{
|
||||
boolean DEFAULT_ASYNC = false;
|
||||
String DEFAULT_OFFSET_FETCH_PERIOD = "PT30S";
|
||||
int DEFAULT_CHAT_RETRIES = 8;
|
||||
String DEFAULT_HTTP_TIMEOUT = "PT10S";
|
||||
|
@ -37,6 +38,8 @@ public interface SeekableStreamSupervisorTuningConfig
|
|||
return (period == null ? new Period(theDefault) : period).toStandardDuration();
|
||||
}
|
||||
|
||||
boolean getChatAsync();
|
||||
|
||||
@JsonProperty
|
||||
Integer getWorkerThreads();
|
||||
|
||||
|
|
|
@ -0,0 +1,667 @@
|
|||
/*
|
||||
* 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.google.common.base.Optional;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import it.unimi.dsi.fastutil.bytes.ByteArrays;
|
||||
import org.apache.druid.indexer.TaskLocation;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
|
||||
import org.apache.druid.rpc.HttpResponseException;
|
||||
import org.apache.druid.rpc.MockServiceClient;
|
||||
import org.apache.druid.rpc.RequestBuilder;
|
||||
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.ServiceNotAvailableException;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionReport;
|
||||
import org.easymock.EasyMock;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.hamcrest.MatcherAssert;
|
||||
import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
|
||||
import org.jboss.netty.handler.codec.http.HttpMethod;
|
||||
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
|
||||
import org.jboss.netty.handler.codec.http.HttpVersion;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
public class SeekableStreamIndexTaskClientAsyncImplTest
|
||||
{
|
||||
private static final String DATASOURCE = "the-datasource";
|
||||
private static final String TASK_ID = "the-task";
|
||||
private static final int MAX_ATTEMPTS = 2;
|
||||
|
||||
private final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
private final Duration httpTimeout = Duration.standardSeconds(1);
|
||||
|
||||
private MockServiceClient serviceClient;
|
||||
private ServiceClientFactory serviceClientFactory;
|
||||
private SeekableStreamIndexTaskClient<Integer, Long> client;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
serviceClient = new MockServiceClient();
|
||||
serviceClientFactory = (serviceName, serviceLocator, retryPolicy) -> {
|
||||
Assert.assertEquals(TASK_ID, serviceName);
|
||||
return serviceClient;
|
||||
};
|
||||
client = new TestSeekableStreamIndexTaskClientAsyncImpl();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown()
|
||||
{
|
||||
serviceClient.verify();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getCheckpointsAsync() throws Exception
|
||||
{
|
||||
final Map<Integer, Map<Integer, Long>> checkpoints = ImmutableMap.of(0, ImmutableMap.of(2, 3L));
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/checkpoints").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(checkpoints)
|
||||
);
|
||||
|
||||
Assert.assertEquals(checkpoints, client.getCheckpointsAsync(TASK_ID, false).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getCurrentOffsetsAsync() throws Exception
|
||||
{
|
||||
final ImmutableMap<Integer, Long> offsets = ImmutableMap.of(2, 3L);
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/offsets/current").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(offsets)
|
||||
);
|
||||
|
||||
Assert.assertEquals(offsets, client.getCurrentOffsetsAsync(TASK_ID, false).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getEndOffsetsAsync() throws Exception
|
||||
{
|
||||
final ImmutableMap<Integer, Long> offsets = ImmutableMap.of(2, 3L);
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/offsets/end").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(offsets)
|
||||
);
|
||||
|
||||
Assert.assertEquals(offsets, client.getEndOffsetsAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getEndOffsetsAsync_notAvailable() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/offsets/end").timeout(httpTimeout),
|
||||
new ServiceNotAvailableException(TASK_ID)
|
||||
);
|
||||
|
||||
Assert.assertEquals(Collections.emptyMap(), client.getEndOffsetsAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_stopAsync_publish_ok() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/stop?publish=true").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
);
|
||||
|
||||
Assert.assertEquals(true, client.stopAsync(TASK_ID, true).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_stopAsync_noPublish_ok() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
);
|
||||
|
||||
Assert.assertEquals(true, client.stopAsync(TASK_ID, false).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_stopAsync_noPublish_httpError() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout),
|
||||
new HttpResponseException(
|
||||
new StringFullResponseHolder(
|
||||
new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.SERVICE_UNAVAILABLE),
|
||||
StandardCharsets.UTF_8
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertEquals(false, client.stopAsync(TASK_ID, false).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_stopAsync_noPublish_notAvailable() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout),
|
||||
new ServiceNotAvailableException(TASK_ID)
|
||||
);
|
||||
|
||||
Assert.assertEquals(false, client.stopAsync(TASK_ID, false).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_stopAsync_noPublish_closed() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout),
|
||||
new ServiceClosedException(TASK_ID)
|
||||
);
|
||||
|
||||
Assert.assertEquals(true, client.stopAsync(TASK_ID, false).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_stopAsync_noPublish_ioException()
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout),
|
||||
new IOException()
|
||||
);
|
||||
|
||||
final ExecutionException e = Assert.assertThrows(
|
||||
ExecutionException.class,
|
||||
() -> client.stopAsync(TASK_ID, false).get()
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IOException.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_resumeAsync_ok() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/resume").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
);
|
||||
|
||||
Assert.assertEquals(true, client.resumeAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_resumeAsync_ioException() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/resume").timeout(httpTimeout),
|
||||
new IOException()
|
||||
);
|
||||
|
||||
Assert.assertEquals(false, client.resumeAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_setEndOffsetsAsync() throws Exception
|
||||
{
|
||||
final Map<Integer, Long> offsets = ImmutableMap.of(1, 3L);
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/offsets/end?finish=false")
|
||||
.content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(offsets))
|
||||
.timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
);
|
||||
|
||||
Assert.assertEquals(true, client.setEndOffsetsAsync(TASK_ID, offsets, false).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_setEndOffsetsAsync_ioException() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/resume").timeout(httpTimeout),
|
||||
new IOException()
|
||||
);
|
||||
|
||||
Assert.assertEquals(false, client.resumeAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getStatusAsync() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.READING)
|
||||
);
|
||||
|
||||
Assert.assertEquals(SeekableStreamIndexTaskRunner.Status.READING, client.getStatusAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getStatusAsync_notAvailable() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout),
|
||||
new ServiceNotAvailableException(TASK_ID)
|
||||
);
|
||||
|
||||
Assert.assertEquals(SeekableStreamIndexTaskRunner.Status.NOT_STARTED, client.getStatusAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getStartTimeAsync() throws Exception
|
||||
{
|
||||
final DateTime startTime = DateTimes.of("2000");
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/time/start").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(startTime)
|
||||
);
|
||||
|
||||
Assert.assertEquals(startTime, client.getStartTimeAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getStartTimeAsync_noContent() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/time/start").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
);
|
||||
|
||||
Assert.assertNull(client.getStartTimeAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getStartTimeAsync_notAvailable() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/time/start").timeout(httpTimeout),
|
||||
new ServiceNotAvailableException(TASK_ID)
|
||||
);
|
||||
|
||||
Assert.assertNull(client.getStartTimeAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_pauseAsync_immediateOk() throws Exception
|
||||
{
|
||||
final Map<Integer, Long> offsets = ImmutableMap.of(1, 3L);
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(offsets)
|
||||
);
|
||||
|
||||
Assert.assertEquals(offsets, client.pauseAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_pauseAsync_immediateBadStatus() throws Exception
|
||||
{
|
||||
final Map<Integer, Long> offsets = ImmutableMap.of(1, 3L);
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout),
|
||||
HttpResponseStatus.CONTINUE,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(offsets)
|
||||
);
|
||||
|
||||
final ExecutionException e = Assert.assertThrows(
|
||||
ExecutionException.class,
|
||||
() -> client.pauseAsync(TASK_ID).get()
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IllegalStateException.class));
|
||||
MatcherAssert.assertThat(
|
||||
e.getCause().getMessage(),
|
||||
CoreMatchers.startsWith("Pause request for task [the-task] failed with response [100 Continue]")
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_pauseAsync_oneIteration() throws Exception
|
||||
{
|
||||
final Map<Integer, Long> offsets = ImmutableMap.of(1, 3L);
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout),
|
||||
HttpResponseStatus.ACCEPTED,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
).expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.PAUSED)
|
||||
).expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/offsets/current").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(offsets)
|
||||
);
|
||||
|
||||
Assert.assertEquals(offsets, client.pauseAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_pauseAsync_oneIterationWithError()
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout),
|
||||
HttpResponseStatus.ACCEPTED,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
).expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout),
|
||||
new IOException()
|
||||
);
|
||||
|
||||
final ExecutionException e = Assert.assertThrows(
|
||||
ExecutionException.class,
|
||||
() -> client.pauseAsync(TASK_ID).get()
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IOException.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_pauseAsync_twoIterations() throws Exception
|
||||
{
|
||||
final Map<Integer, Long> offsets = ImmutableMap.of(1, 3L);
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout),
|
||||
HttpResponseStatus.ACCEPTED,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
).expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.READING)
|
||||
).expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.PAUSED)
|
||||
).expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/offsets/current").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(offsets)
|
||||
);
|
||||
|
||||
Assert.assertEquals(offsets, client.pauseAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_pauseAsync_threeIterations() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout),
|
||||
HttpResponseStatus.ACCEPTED,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
).expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.READING)
|
||||
).expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.READING)
|
||||
).expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.READING)
|
||||
);
|
||||
|
||||
final ExecutionException e = Assert.assertThrows(
|
||||
ExecutionException.class,
|
||||
() -> client.pauseAsync(TASK_ID).get()
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IllegalStateException.class));
|
||||
MatcherAssert.assertThat(
|
||||
e.getCause().getMessage(),
|
||||
CoreMatchers.startsWith("Task [the-task] failed to change its status from [READING] to [PAUSED]")
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getMovingAveragesAsync() throws Exception
|
||||
{
|
||||
final Map<String, Object> retVal = ImmutableMap.of("foo", "xyz");
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/rowStats").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(retVal)
|
||||
);
|
||||
|
||||
Assert.assertEquals(retVal, client.getMovingAveragesAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getMovingAveragesAsync_empty() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/rowStats").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
);
|
||||
|
||||
Assert.assertNull(client.getMovingAveragesAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getMovingAveragesAsync_null() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/rowStats").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
null
|
||||
);
|
||||
|
||||
Assert.assertNull(client.getMovingAveragesAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getParseErrorsAsync() throws Exception
|
||||
{
|
||||
final List<ParseExceptionReport> retVal = ImmutableList.of(
|
||||
new ParseExceptionReport("xyz", "foo", Collections.emptyList(), 123L)
|
||||
);
|
||||
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/unparseableEvents").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
jsonMapper.writeValueAsBytes(retVal)
|
||||
);
|
||||
|
||||
Assert.assertEquals(retVal, client.getParseErrorsAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getParseErrorsAsync_empty() throws Exception
|
||||
{
|
||||
serviceClient.expect(
|
||||
new RequestBuilder(HttpMethod.GET, "/unparseableEvents").timeout(httpTimeout),
|
||||
HttpResponseStatus.OK,
|
||||
Collections.emptyMap(),
|
||||
ByteArrays.EMPTY_ARRAY
|
||||
);
|
||||
|
||||
Assert.assertNull(client.getParseErrorsAsync(TASK_ID).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_serviceLocator_unknownTask() throws Exception
|
||||
{
|
||||
final TaskInfoProvider taskInfoProvider = EasyMock.createStrictMock(TaskInfoProvider.class);
|
||||
EasyMock.expect(taskInfoProvider.getTaskStatus(TASK_ID))
|
||||
.andReturn(Optional.absent());
|
||||
EasyMock.replay(taskInfoProvider);
|
||||
|
||||
try (final SeekableStreamIndexTaskClientAsyncImpl.SeekableStreamTaskLocator locator =
|
||||
new SeekableStreamIndexTaskClientAsyncImpl.SeekableStreamTaskLocator(taskInfoProvider, TASK_ID)) {
|
||||
Assert.assertEquals(
|
||||
ServiceLocations.closed(),
|
||||
locator.locate().get()
|
||||
);
|
||||
}
|
||||
|
||||
EasyMock.verify(taskInfoProvider);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_serviceLocator_unknownLocation() throws Exception
|
||||
{
|
||||
final TaskInfoProvider taskInfoProvider = EasyMock.createStrictMock(TaskInfoProvider.class);
|
||||
EasyMock.expect(taskInfoProvider.getTaskStatus(TASK_ID))
|
||||
.andReturn(Optional.of(TaskStatus.running(TASK_ID)));
|
||||
EasyMock.expect(taskInfoProvider.getTaskLocation(TASK_ID))
|
||||
.andReturn(TaskLocation.unknown());
|
||||
EasyMock.replay(taskInfoProvider);
|
||||
|
||||
try (final SeekableStreamIndexTaskClientAsyncImpl.SeekableStreamTaskLocator locator =
|
||||
new SeekableStreamIndexTaskClientAsyncImpl.SeekableStreamTaskLocator(taskInfoProvider, TASK_ID)) {
|
||||
Assert.assertEquals(
|
||||
ServiceLocations.forLocations(Collections.emptySet()),
|
||||
locator.locate().get()
|
||||
);
|
||||
}
|
||||
|
||||
EasyMock.verify(taskInfoProvider);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_serviceLocator_found() throws Exception
|
||||
{
|
||||
final TaskInfoProvider taskInfoProvider = EasyMock.createStrictMock(TaskInfoProvider.class);
|
||||
EasyMock.expect(taskInfoProvider.getTaskStatus(TASK_ID))
|
||||
.andReturn(Optional.of(TaskStatus.running(TASK_ID)));
|
||||
EasyMock.expect(taskInfoProvider.getTaskLocation(TASK_ID))
|
||||
.andReturn(TaskLocation.create("foo", 80, -1));
|
||||
EasyMock.replay(taskInfoProvider);
|
||||
|
||||
try (final SeekableStreamIndexTaskClientAsyncImpl.SeekableStreamTaskLocator locator =
|
||||
new SeekableStreamIndexTaskClientAsyncImpl.SeekableStreamTaskLocator(taskInfoProvider, TASK_ID)) {
|
||||
Assert.assertEquals(
|
||||
ServiceLocations.forLocation(new ServiceLocation("foo", 80, -1, "/druid/worker/v1/chat/" + TASK_ID)),
|
||||
locator.locate().get()
|
||||
);
|
||||
}
|
||||
|
||||
EasyMock.verify(taskInfoProvider);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_serviceLocator_closed() throws Exception
|
||||
{
|
||||
final TaskInfoProvider taskInfoProvider = EasyMock.createStrictMock(TaskInfoProvider.class);
|
||||
EasyMock.expect(taskInfoProvider.getTaskStatus(TASK_ID))
|
||||
.andReturn(Optional.of(TaskStatus.success(TASK_ID)));
|
||||
EasyMock.replay(taskInfoProvider);
|
||||
|
||||
try (final SeekableStreamIndexTaskClientAsyncImpl.SeekableStreamTaskLocator locator =
|
||||
new SeekableStreamIndexTaskClientAsyncImpl.SeekableStreamTaskLocator(taskInfoProvider, TASK_ID)) {
|
||||
Assert.assertEquals(
|
||||
ServiceLocations.closed(),
|
||||
locator.locate().get()
|
||||
);
|
||||
}
|
||||
|
||||
EasyMock.verify(taskInfoProvider);
|
||||
}
|
||||
|
||||
private class TestSeekableStreamIndexTaskClientAsyncImpl extends SeekableStreamIndexTaskClientAsyncImpl<Integer, Long>
|
||||
{
|
||||
public TestSeekableStreamIndexTaskClientAsyncImpl()
|
||||
{
|
||||
super(DATASOURCE, serviceClientFactory, null, jsonMapper, httpTimeout, MAX_ATTEMPTS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<Integer> getPartitionType()
|
||||
{
|
||||
return Integer.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<Long> getSequenceType()
|
||||
{
|
||||
return Long.class;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -447,6 +447,12 @@ public class SeekableStreamSupervisorSpecTest extends EasyMockSupport
|
|||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getChatAsync()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getChatThreads()
|
||||
{
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.base.Optional;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.errorprone.annotations.concurrent.GuardedBy;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
|
@ -152,17 +153,17 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
|
|||
EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes();
|
||||
|
||||
EasyMock.expect(taskClientFactory.build(
|
||||
EasyMock.anyObject(),
|
||||
EasyMock.anyString(),
|
||||
EasyMock.anyInt(),
|
||||
EasyMock.anyObject(),
|
||||
EasyMock.anyLong()
|
||||
EasyMock.anyInt(),
|
||||
EasyMock.anyObject()
|
||||
)).andReturn(
|
||||
indexTaskClient).anyTimes();
|
||||
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
|
||||
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
|
||||
|
||||
taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
|
||||
EasyMock.expectLastCall().times(0, 1);
|
||||
|
||||
EasyMock
|
||||
.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(null).anyTimes();
|
||||
|
@ -971,6 +972,53 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
|
|||
verifyAll();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetStats()
|
||||
{
|
||||
EasyMock.expect(spec.isSuspended()).andReturn(false);
|
||||
EasyMock.expect(indexTaskClient.getMovingAveragesAsync("task1"))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of("prop1", "val1")))
|
||||
.times(1);
|
||||
EasyMock.expect(indexTaskClient.getMovingAveragesAsync("task2"))
|
||||
.andReturn(Futures.immediateFuture(ImmutableMap.of("prop2", "val2")))
|
||||
.times(1);
|
||||
replayAll();
|
||||
|
||||
final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor();
|
||||
|
||||
supervisor.start();
|
||||
supervisor.addTaskGroupToActivelyReadingTaskGroup(
|
||||
supervisor.getTaskGroupIdForPartition("0"),
|
||||
ImmutableMap.of("0", "0"),
|
||||
Optional.absent(),
|
||||
Optional.absent(),
|
||||
ImmutableSet.of("task1"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
supervisor.addTaskGroupToPendingCompletionTaskGroup(
|
||||
supervisor.getTaskGroupIdForPartition("1"),
|
||||
ImmutableMap.of("1", "0"),
|
||||
Optional.absent(),
|
||||
Optional.absent(),
|
||||
ImmutableSet.of("task2"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
Map<String, Map<String, Object>> stats = supervisor.getStats();
|
||||
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(1, stats.size());
|
||||
Assert.assertEquals(ImmutableSet.of("0"), stats.keySet());
|
||||
Assert.assertEquals(
|
||||
ImmutableMap.of(
|
||||
"task1", ImmutableMap.of("prop1", "val1"),
|
||||
"task2", ImmutableMap.of("prop2", "val2")
|
||||
),
|
||||
stats.get("0")
|
||||
);
|
||||
}
|
||||
|
||||
private List<Event> filterMetrics(List<Event> events, List<String> whitelist)
|
||||
{
|
||||
List<Event> result = events.stream()
|
||||
|
@ -1093,6 +1141,12 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
|
|||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getChatAsync()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getChatThreads()
|
||||
{
|
||||
|
|
|
@ -183,4 +183,17 @@ public class RequestBuilder
|
|||
result = 31 * result + Arrays.hashCode(content);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "RequestBuilder{" +
|
||||
"method=" + method +
|
||||
", encodedPathAndQueryString='" + encodedPathAndQueryString + '\'' +
|
||||
", headers=" + headers +
|
||||
", contentType='" + contentType + '\'' +
|
||||
", content=" + Arrays.toString(content) +
|
||||
", timeout=" + timeout +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,8 +52,9 @@ public interface ServiceClient
|
|||
* attempts. Redirects are validated against the targets returned by {@link ServiceLocator}: the client will not
|
||||
* follow a redirect to a target that does not appear in the returned {@link ServiceLocations}.
|
||||
*
|
||||
* If the service is unavailable at the time an attempt is made -- i.e. if {@link ServiceLocator#locate()} returns an
|
||||
* empty set -- then an attempt is consumed and the client will try to locate the service again on the next attempt.
|
||||
* If the service is unavailable at the time an attempt is made, the client will automatically retry based on
|
||||
* {@link ServiceRetryPolicy#retryNotAvailable()}. If true, an attempt is consumed and the client will try to locate
|
||||
* the service again on the next attempt. If false, the call immediately returns {@link ServiceNotAvailableException}.
|
||||
*
|
||||
* If an exception occurs midstream after an OK HTTP response (2xx) then the behavior depends on the handler. If
|
||||
* the handler has not yet returned a finished object, the service client will automatically retry based on the
|
||||
|
|
|
@ -116,10 +116,10 @@ public class ServiceClientImpl implements ServiceClient
|
|||
final long nextAttemptNumber = attemptNumber + 1;
|
||||
|
||||
if (serviceLocation == null) {
|
||||
// Null location means the service is not currently available. Trigger a retry.
|
||||
final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber);
|
||||
// Null location means the service is not currently available. Trigger a retry, if retryable.
|
||||
if (retryPolicy.retryNotAvailable() && shouldTry(nextAttemptNumber)) {
|
||||
final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber);
|
||||
|
||||
if (shouldTry(nextAttemptNumber)) {
|
||||
log.info(
|
||||
"Service [%s] not available on attempt #%d; retrying in %,d ms.",
|
||||
serviceName,
|
||||
|
|
|
@ -53,4 +53,9 @@ public interface ServiceRetryPolicy
|
|||
* Returns whether the given exception can be retried.
|
||||
*/
|
||||
boolean retryThrowable(Throwable t);
|
||||
|
||||
/**
|
||||
* Returns whether service-not-available, i.e. empty {@link ServiceLocations#getLocations()}, can be retried.
|
||||
*/
|
||||
boolean retryNotAvailable();
|
||||
}
|
||||
|
|
|
@ -43,12 +43,14 @@ public class StandardRetryPolicy implements ServiceRetryPolicy
|
|||
private final long maxAttempts;
|
||||
private final long minWaitMillis;
|
||||
private final long maxWaitMillis;
|
||||
private final boolean retryNotAvailable;
|
||||
|
||||
private StandardRetryPolicy(long maxAttempts, long minWaitMillis, long maxWaitMillis)
|
||||
private StandardRetryPolicy(long maxAttempts, long minWaitMillis, long maxWaitMillis, boolean retryNotAvailable)
|
||||
{
|
||||
this.maxAttempts = maxAttempts;
|
||||
this.minWaitMillis = minWaitMillis;
|
||||
this.maxWaitMillis = maxWaitMillis;
|
||||
this.retryNotAvailable = retryNotAvailable;
|
||||
|
||||
if (maxAttempts == 0) {
|
||||
throw new IAE("maxAttempts must be positive (limited) or negative (unlimited); cannot be zero.");
|
||||
|
@ -109,11 +111,18 @@ public class StandardRetryPolicy implements ServiceRetryPolicy
|
|||
|| (t.getCause() != null && retryThrowable(t.getCause()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean retryNotAvailable()
|
||||
{
|
||||
return retryNotAvailable;
|
||||
}
|
||||
|
||||
public static class Builder
|
||||
{
|
||||
private long maxAttempts = 0; // Zero is an invalid value: so, this parameter must be explicitly specified
|
||||
private long minWaitMillis = DEFAULT_MIN_WAIT_MS;
|
||||
private long maxWaitMillis = DEFAULT_MAX_WAIT_MS;
|
||||
private boolean retryNotAvailable = true;
|
||||
|
||||
public Builder maxAttempts(final long maxAttempts)
|
||||
{
|
||||
|
@ -133,9 +142,15 @@ public class StandardRetryPolicy implements ServiceRetryPolicy
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder retryNotAvailable(final boolean retryNotAvailable)
|
||||
{
|
||||
this.retryNotAvailable = retryNotAvailable;
|
||||
return this;
|
||||
}
|
||||
|
||||
public StandardRetryPolicy build()
|
||||
{
|
||||
return new StandardRetryPolicy(maxAttempts, minWaitMillis, maxWaitMillis);
|
||||
return new StandardRetryPolicy(maxAttempts, minWaitMillis, maxWaitMillis, retryNotAvailable);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -77,6 +77,12 @@ public class SpecificTaskRetryPolicy implements ServiceRetryPolicy
|
|||
return StandardRetryPolicy.unlimited().retryThrowable(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean retryNotAvailable()
|
||||
{
|
||||
return baseRetryPolicy.retryNotAvailable();
|
||||
}
|
||||
|
||||
private boolean isTaskMismatch(final HttpResponse response)
|
||||
{
|
||||
// See class-level javadocs for details on why we do this.
|
||||
|
|
|
@ -0,0 +1,128 @@
|
|||
/*
|
||||
* 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.rpc;
|
||||
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.java.util.common.Either;
|
||||
import org.apache.druid.java.util.http.client.response.ClientResponse;
|
||||
import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
|
||||
import org.jboss.netty.buffer.ChannelBuffers;
|
||||
import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
|
||||
import org.jboss.netty.handler.codec.http.HttpResponse;
|
||||
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
|
||||
import org.jboss.netty.handler.codec.http.HttpVersion;
|
||||
import org.junit.Assert;
|
||||
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
|
||||
/**
|
||||
* Mock implementation of {@link ServiceClient}.
|
||||
*/
|
||||
public class MockServiceClient implements ServiceClient
|
||||
{
|
||||
private final Queue<Expectation> expectations = new ArrayDeque<>(16);
|
||||
|
||||
@Override
|
||||
public <IntermediateType, FinalType> ListenableFuture<FinalType> asyncRequest(
|
||||
final RequestBuilder requestBuilder,
|
||||
final HttpResponseHandler<IntermediateType, FinalType> handler
|
||||
)
|
||||
{
|
||||
final Expectation expectation = expectations.poll();
|
||||
|
||||
Assert.assertEquals(
|
||||
"request",
|
||||
expectation == null ? null : expectation.request,
|
||||
requestBuilder
|
||||
);
|
||||
|
||||
if (expectation.response.isValue()) {
|
||||
final ClientResponse<FinalType> response =
|
||||
handler.done(handler.handleResponse(expectation.response.valueOrThrow(), chunkNum -> 0));
|
||||
return Futures.immediateFuture(response.getObj());
|
||||
} else {
|
||||
return Futures.immediateFailedFuture(expectation.response.error());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServiceClient withRetryPolicy(final ServiceRetryPolicy retryPolicy)
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
public MockServiceClient expect(final RequestBuilder request, final HttpResponse response)
|
||||
{
|
||||
expectations.add(new Expectation(request, Either.value(response)));
|
||||
return this;
|
||||
}
|
||||
|
||||
public MockServiceClient expect(
|
||||
final RequestBuilder request,
|
||||
final HttpResponseStatus status,
|
||||
final Map<String, String> headers,
|
||||
final byte[] content
|
||||
)
|
||||
{
|
||||
final HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, status);
|
||||
for (Map.Entry<String, String> headerEntry : headers.entrySet()) {
|
||||
response.headers().set(headerEntry.getKey(), headerEntry.getValue());
|
||||
}
|
||||
if (content != null) {
|
||||
response.setContent(ChannelBuffers.wrappedBuffer(content));
|
||||
}
|
||||
return expect(request, response);
|
||||
}
|
||||
|
||||
public MockServiceClient expect(final RequestBuilder request, final Throwable e)
|
||||
{
|
||||
expectations.add(new Expectation(request, Either.error(e)));
|
||||
return this;
|
||||
}
|
||||
|
||||
public void verify()
|
||||
{
|
||||
Assert.assertTrue("all requests were made", expectations.isEmpty());
|
||||
}
|
||||
|
||||
private static class Expectation
|
||||
{
|
||||
private final RequestBuilder request;
|
||||
private final Either<Throwable, HttpResponse> response;
|
||||
|
||||
public Expectation(RequestBuilder request, Either<Throwable, HttpResponse> response)
|
||||
{
|
||||
this.request = request;
|
||||
this.response = response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "Expectation{" +
|
||||
"request=" + request +
|
||||
", response=" + response +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
}
|
|
@ -406,6 +406,33 @@ public class ServiceClientImplTest
|
|||
Assert.assertEquals(expectedResponseObject, response);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_request_serviceUnavailableNoRetry()
|
||||
{
|
||||
final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo");
|
||||
|
||||
// Service unavailable.
|
||||
stubLocatorCall(locations());
|
||||
|
||||
serviceClient = makeServiceClient(
|
||||
StandardRetryPolicy.builder()
|
||||
.retryNotAvailable(false)
|
||||
.maxAttempts(ServiceRetryPolicy.UNLIMITED)
|
||||
.build()
|
||||
);
|
||||
|
||||
final ExecutionException e = Assert.assertThrows(
|
||||
ExecutionException.class,
|
||||
() -> doRequest(serviceClient, requestBuilder)
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(ServiceNotAvailableException.class));
|
||||
MatcherAssert.assertThat(
|
||||
e.getCause(),
|
||||
ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Service [test-service] is not available"))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_request_serviceClosed()
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue