diff --git a/core/src/main/java/org/apache/druid/common/guava/FutureUtils.java b/core/src/main/java/org/apache/druid/common/guava/FutureUtils.java index 73d4f3ab15e..cf9a7f0ef71 100644 --- a/core/src/main/java/org/apache/druid/common/guava/FutureUtils.java +++ b/core/src/main/java/org/apache/druid/common/guava/FutureUtils.java @@ -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) v -> ...)} + */ + public static ListenableFuture transformAsync(final ListenableFuture future, final AsyncFunction 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 ListenableFuture>> coalesce(final List> futures) + { + return transform( + Futures.successfulAsList(futures), + values -> { + final List> eithers = new ArrayList<>(); + + for (int i = 0; i < values.size(); i++) { + final ListenableFuture 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 diff --git a/core/src/main/java/org/apache/druid/java/util/common/Either.java b/core/src/main/java/org/apache/druid/java/util/common/Either.java index 71412ba4066..5a2720d0748 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/Either.java +++ b/core/src/main/java/org/apache/druid/java/util/common/Either.java @@ -62,6 +62,11 @@ public class Either 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 /** * 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() diff --git a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java index d51c042e91d..bfb1dc79d78 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java @@ -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 readValue(ObjectMapper mapper, byte[] bytes, Class valueClass) + public static T readValue(ObjectMapper mapper, byte[] bytes, Class 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 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 readValue(ObjectMapper mapper, byte[] bytes, TypeReference valueType) + { + try { + return mapper.readValue(bytes, valueType); } catch (IOException e) { throw new RuntimeException(e); diff --git a/core/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java b/core/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java index 317fa3e56da..416c97e1840 100644 --- a/core/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java +++ b/core/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java @@ -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> 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> 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> futures = new ArrayList<>(); + + futures.add(Futures.immediateFuture("foo")); + futures.add(Futures.immediateCancelledFuture()); + futures.add(Futures.immediateFuture(null)); + + final List> 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> futures = new ArrayList<>(); + final SettableFuture unresolvedFuture = SettableFuture.create(); + + futures.add(Futures.immediateFuture("foo")); + futures.add(unresolvedFuture); + futures.add(Futures.immediateFuture(null)); + + final ListenableFuture>> coalesced = FutureUtils.coalesce(futures); + + Assert.assertThrows( + TimeoutException.class, + () -> coalesced.get(10, TimeUnit.MILLISECONDS) + ); + } + + @Test + public void test_coalesce_cancel() + { + final List> futures = new ArrayList<>(); + final SettableFuture unresolvedFuture = SettableFuture.create(); + + futures.add(Futures.immediateFuture("foo")); + futures.add(unresolvedFuture); + futures.add(Futures.immediateFuture(null)); + + final ListenableFuture>> 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 { diff --git a/core/src/test/java/org/apache/druid/common/jackson/JacksonUtilsTest.java b/core/src/test/java/org/apache/druid/common/jackson/JacksonUtilsTest.java index b08ac2c1ef2..5e70962ef8a 100644 --- a/core/src/test/java/org/apache/druid/common/jackson/JacksonUtilsTest.java +++ b/core/src/test/java/org/apache/druid/common/jackson/JacksonUtilsTest.java @@ -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 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; diff --git a/docs/development/extensions-core/kafka-supervisor-reference.md b/docs/development/extensions-core/kafka-supervisor-reference.md index 6f94f90ffc7..210207302f2 100644 --- a/docs/development/extensions-core/kafka-supervisor-reference.md +++ b/docs/development/extensions-core/kafka-supervisor-reference.md @@ -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).

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.

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.

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) | diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md index 916ea911d3c..84036665ec9 100644 --- a/docs/development/extensions-core/kinesis-ingestion.md +++ b/docs/development/extensions-core/kinesis-ingestion.md @@ -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.

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.

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)| diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java deleted file mode 100644 index a915f44e09d..00000000000 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java +++ /dev/null @@ -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 -{ - 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 getPartitionType() - { - return Integer.class; - } - - @Override - protected Class getSequenceType() - { - return Long.class; - } -} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java index cfa7c723655..091e5cf3ba6 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java @@ -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 +@LazySingleton +public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory { @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 getPartitionType() { - return new KafkaIndexTaskClient( - getHttpClient(), - getMapper(), - taskInfoProvider, - dataSource, - numThreads, - httpTimeout, - numRetries - ); + return Integer.class; + } + + @Override + public Class getSequenceType() + { + return Long.class; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java index 5c7b9e3f6a6..67f18c10f3a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java @@ -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>() - { - } - ).to(KafkaIndexTaskClientFactory.class).in(LazySingleton.class); + // Nothing to do. } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 596e5329240..c99c36b8d45 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -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() diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java index f8429d4d487..9208e99dccd 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -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 client; @Parameterized.Parameters(name = "numThreads = {0}") public static Iterable 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 results = client.getCurrentOffsets(TEST_ID, true); + Map results = client.getCurrentOffsetsAsync(TEST_ID, true).get(); verifyAll(); Assert.assertEquals(0, results.size()); @@ -267,7 +277,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport ); replayAll(); - Map results = client.getCurrentOffsets(TEST_ID, true); + Map results = client.getCurrentOffsetsAsync(TEST_ID, true).get(); verifyAll(); Request request = captured.getValue(); @@ -308,7 +318,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport replayAll(); - Map results = client.getCurrentOffsets(TEST_ID, true); + Map 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 results = client.getEndOffsets(TEST_ID); + Map 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 results = client.pause(TEST_ID); + Map 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 results = client.pause(TEST_ID); + Map 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 { 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 getPartitionType() + { + return Integer.class; + } + + @Override + public Class getSequenceType() + { + return Long.class; } } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 2c5bad93cb5..6e3de31b754 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -143,6 +143,7 @@ public class KafkaIndexTaskTuningConfigTest null, null, null, + null, null ); KafkaIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 2b95135c367..ab14f4dac7d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -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 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 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> 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> 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> 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> 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 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 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 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; } }; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java deleted file mode 100644 index 8f1c66a1acd..00000000000 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java +++ /dev/null @@ -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 -{ - 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 getPartitionType() - { - return String.class; - } - - @Override - protected Class getSequenceType() - { - return String.class; - } - -} - diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java index 6d443464da6..3e4660086f3 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java @@ -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 +@LazySingleton +public class KinesisIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory { @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 getPartitionType() { - return new KinesisIndexTaskClient( - getHttpClient(), - getMapper(), - taskInfoProvider, - dataSource, - numThreads, - httpTimeout, - numRetries - ); + return String.class; + } + + @Override + public Class getSequenceType() + { + return String.class; } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java index bd11e4a2df6..bbbb856a4fa 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java @@ -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>() - { - } - ).to(KinesisIndexTaskClientFactory.class).in(LazySingleton.class); - JsonConfigProvider.bind(binder, PROPERTY_BASE, AWSCredentialsConfig.class, Names.named(AWS_SCOPE)); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 3df3bed1629..94b7b2bdb3b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -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() diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java index c6e56a29718..56ebd1d380d 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java @@ -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 client; @Parameterized.Parameters(name = "numThreads = {0}") public static Iterable 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 results = client.getCurrentOffsets(TEST_ID, true); + Map results = client.getCurrentOffsetsAsync(TEST_ID, true).get(); verifyAll(); Assert.assertEquals(0, results.size()); @@ -268,7 +278,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport ); replayAll(); - Map results = client.getCurrentOffsets(TEST_ID, true); + Map results = client.getCurrentOffsetsAsync(TEST_ID, true).get(); verifyAll(); Request request = captured.getValue(); @@ -309,7 +319,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport replayAll(); - Map results = client.getCurrentOffsets(TEST_ID, true); + Map 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 results = client.getEndOffsets(TEST_ID); + Map 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 results = client.pause(TEST_ID); + Map results = client.pauseAsync(TEST_ID).get(); verifyAll(); Request request = captured.getValue(); @@ -512,7 +524,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport replayAll(); - Map results = client.pause(TEST_ID); + Map 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 { 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 getPartitionType() + { + return String.class; + } + + @Override + public Class getSequenceType() + { + return String.class; } } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index acb599856ae..823c741e356 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -299,6 +299,7 @@ public class KinesisIndexTaskTuningConfigTest null, null, null, + null, 1000, 500, 500, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 530009302c0..50fbe2f721d 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -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 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> 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> 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> 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> 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 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 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 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 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; } }; diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 03faee8f12d..cba1cace708 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 druid-indexing-service @@ -382,6 +383,18 @@ + + org.jacoco + jacoco-maven-plugin + ${jacoco.version} + + + + org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientSyncImpl.class + + + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java index 21de9f21244..4334fcb35c7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java @@ -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 diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 5bfac7338d7..18631626d0f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -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 extends IndexTaskClient +public interface SeekableStreamIndexTaskClient { - private static final TypeReference> TYPE_REFERENCE_LIST_PARSE_EXCEPTION_REPORT = - new TypeReference>() - { - }; + TypeReference> TYPE_REFERENCE_LIST_PARSE_EXCEPTION_REPORT = + new TypeReference>() {}; - 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>> 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 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 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 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> 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 setEndOffsetsAsync( + String id, + Map endOffsets, + boolean finalize + ); - public Map 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> 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> 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> 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> 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 getStatusAsync(String id); - public SeekableStreamIndexTaskRunner.Status getStatus(final String id) - { - log.debug("GetStatus task[%s]", id); + Class 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 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 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 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 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> 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>> getCheckpointsAsync( - final String id, - final boolean retry - ) - { - return doAsync(() -> getCheckpoints(id, retry)); - } - - public Map 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 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 stopAsync(final String id, final boolean publish) - { - return doAsync(() -> stop(id, publish)); - } - - - public ListenableFuture resumeAsync(final String id) - { - return doAsync(() -> resume(id)); - } - - - public ListenableFuture getStartTimeAsync(final String id) - { - return doAsync(() -> getStartTime(id)); - } - - - public ListenableFuture> pauseAsync(final String id) - { - return doAsync(() -> pause(id)); - } - - public ListenableFuture setEndOffsetsAsync( - final String id, - final Map endOffsets, - final boolean finalize - ) - { - return doAsync(() -> setEndOffsets(id, endOffsets, finalize)); - } - - public ListenableFuture> getCurrentOffsetsAsync( - final String id, - final boolean retry - ) - { - return doAsync(() -> getCurrentOffsets(id, retry)); - } - - public ListenableFuture> getEndOffsetsAsync(final String id) - { - return doAsync(() -> getEndOffsets(id)); - } - - - public ListenableFuture> getMovingAveragesAsync(final String id) - { - return doAsync(() -> getMovingAverages(id)); - } - - public ListenableFuture> getParseErrorsAsync(final String id) - { - return doAsync(() -> getParseErrors(id)); - } - - public ListenableFuture getStatusAsync(final String id) - { - return doAsync(() -> getStatus(id)); - } - - protected abstract Class getPartitionType(); - - protected abstract Class getSequenceType(); + void close(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java new file mode 100644 index 00000000000..f124ca25ca4 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -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 + implements SeekableStreamIndexTaskClient +{ + 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>> 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>) + 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 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 resumeAsync(final String id) + { + return makeRequest(id, new RequestBuilder(HttpMethod.POST, "/resume")) + .onSuccess(r -> true) + .onException(e -> Either.value(false)) + .go(); + } + + @Override + public ListenableFuture> 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> 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 setEndOffsetsAsync( + final String id, + final Map 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 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 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> pauseAsync(String id) + { + final ListenableFuture> 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> 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> 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 makeRequest( + String taskId, + RequestBuilder requestBuilder + ) + { + return new SeekableStreamRequestBuilder<>( + taskId, + requestBuilder, + IgnoreHttpResponseHandler.INSTANCE, + Function.identity() + ); + } + + /** + * Helper for deserializing offset maps. + */ + private Map 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> getOffsetsWhenPaused( + final String taskId, + final RetryPolicy retryPolicy + ) + { + final ListenableFuture 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> retVal = SettableFuture.create(); + retryExec.schedule( + () -> + Futures.addCallback( + getOffsetsWhenPaused(taskId, retryPolicy), + new FutureCallback>() + { + @Override + public void onSuccess(@Nullable Map 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 + { + private final String taskId; + private final RequestBuilder requestBuilder; + + private final List>> exceptionMappers = new ArrayList<>(); + private HttpResponseHandler responseHandler; + private Function responseTransformer; + private boolean retry = true; + + SeekableStreamRequestBuilder( + String taskId, + RequestBuilder requestBuilder, + HttpResponseHandler responseHandler, + Function 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 SeekableStreamRequestBuilder handler( + final HttpResponseHandler handler + ) + { + this.responseHandler = (HttpResponseHandler) handler; + return (SeekableStreamRequestBuilder) this; + } + + /** + * Response mapping for successful requests. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public SeekableStreamRequestBuilder onSuccess( + final Function responseTransformer + ) + { + this.responseTransformer = (Function) responseTransformer; + return (SeekableStreamRequestBuilder) this; + } + + /** + * Whether the request should be retried on failure. Default is true. + */ + public SeekableStreamRequestBuilder retry(boolean retry) + { + this.retry = retry; + return this; + } + + /** + * Error mapping for all exceptions. + */ + public SeekableStreamRequestBuilder onException(final Function> fn) + { + exceptionMappers.add(fn); + return this; + } + + /** + * Error mapping for {@link HttpResponseException}, which occurs when a task returns a non-2xx HTTP code. + */ + public SeekableStreamRequestBuilder onHttpError(final Function> 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 onNotAvailable(final Function> 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 onClosed(final Function> fn) + { + return onException(e -> { + if (e instanceof ServiceClosedException) { + return fn.apply((ServiceClosedException) e); + } else { + return Either.error(e); + } + }); + } + + /** + * Issue the request. + */ + public ListenableFuture go() + { + final ServiceClient client = makeClient(taskId, retry); + final SettableFuture retVal = SettableFuture.create(); + + Futures.addCallback( + FutureUtils.transform( + client.asyncRequest(requestBuilder.timeout(httpTimeout), responseHandler), + responseTransformer + ), + new FutureCallback() + { + @Override + public void onSuccess(@Nullable T result) + { + retVal.set(result); + } + + @Override + public void onFailure(Throwable t) + { + Either either = Either.error(t); + + for (final Function> exceptionMapper : exceptionMappers) { + if (!either.isError()) { + break; + } + + try { + final Either 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 locate() + { + final Optional 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. + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java index 6a909704bda..a30954166a5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java @@ -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 - implements IndexTaskClientFactory +public abstract class SeekableStreamIndexTaskClientFactory { - 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 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 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( + dataSource, + serviceClientFactory, + taskInfoProvider, + jsonMapper, + tuningConfig.getHttpTimeout(), + tuningConfig.getChatRetries() + ) + { + @Override + public Class getPartitionType() + { + return SeekableStreamIndexTaskClientFactory.this.getPartitionType(); + } + + @Override + public Class getSequenceType() + { + return SeekableStreamIndexTaskClientFactory.this.getSequenceType(); + } + }; } + + private SeekableStreamIndexTaskClient 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( + httpClient, + jsonMapper, + taskInfoProvider, + dataSource, + chatThreads, + tuningConfig.getHttpTimeout(), + tuningConfig.getChatRetries() + ) + { + @Override + public Class getPartitionType() + { + return SeekableStreamIndexTaskClientFactory.this.getPartitionType(); + } + + @Override + public Class getSequenceType() + { + return SeekableStreamIndexTaskClientFactory.this.getSequenceType(); + } + }; + } + + protected abstract Class getPartitionType(); + + protected abstract Class getSequenceType(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientSyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientSyncImpl.java new file mode 100644 index 00000000000..9f01a6f8c95 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientSyncImpl.java @@ -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 + extends IndexTaskClient + implements SeekableStreamIndexTaskClient +{ + 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 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 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 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 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> 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>> getCheckpointsAsync( + final String id, + final boolean retry + ) + { + return doAsync(() -> getCheckpoints(id, retry)); + } + + private Map 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 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 stopAsync(final String id, final boolean publish) + { + return doAsync(() -> stop(id, publish)); + } + + @Override + public ListenableFuture resumeAsync(final String id) + { + return doAsync(() -> resume(id)); + } + + @Override + public ListenableFuture getStartTimeAsync(final String id) + { + return doAsync(() -> getStartTime(id)); + } + + @Override + public ListenableFuture> pauseAsync(final String id) + { + return doAsync(() -> pause(id)); + } + + @Override + public ListenableFuture setEndOffsetsAsync( + final String id, + final Map endOffsets, + final boolean finalize + ) + { + return doAsync(() -> setEndOffsets(id, endOffsets, finalize)); + } + + @Override + public ListenableFuture> getCurrentOffsetsAsync( + final String id, + final boolean retry + ) + { + return doAsync(() -> getCurrentOffsets(id, retry)); + } + + @Override + public ListenableFuture> getEndOffsetsAsync(final String id) + { + return doAsync(() -> getEndOffsets(id)); + } + + @Override + public ListenableFuture> getMovingAveragesAsync(final String id) + { + return doAsync(() -> getMovingAverages(id)); + } + + @Override + public ListenableFuture> getParseErrorsAsync(final String id) + { + return doAsync(() -> getParseErrors(id)); + } + + @Override + public ListenableFuture getStatusAsync(final String id) + { + return doAsync(() -> getStatus(id)); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 26d864e2925..53769f84985 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -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> taskClientFactory, + final SeekableStreamIndexTaskClientFactory taskClientFactory, final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, @@ -789,7 +791,7 @@ public abstract class SeekableStreamSupervisor results = Futures.successfulAsList(futures) - .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + List> results = coalesceAndAwait(futures); for (int i = 0; i < results.size(); i++) { - StatsFromTaskResult result = results.get(i); - if (result != null) { - Map 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 groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> new HashMap<>()); + groupMap.put(result.getTaskId(), result.getStats()); + } } else { Pair 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 parseErrorsTreeSet = new TreeSet<>(PARSE_EXCEPTION_REPORT_COMPARATOR); parseErrorsTreeSet.addAll(lastKnownParseErrors); - List results = Futures.successfulAsList(futures) - .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + List> 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 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() + getStatusAndPossiblyEndOffsets(taskId), + new Function>, Boolean>() { @Override - public Boolean apply(SeekableStreamIndexTaskRunner.Status status) + public Boolean apply(Pair> pair) { + final SeekableStreamIndexTaskRunner.Status status = pair.lhs; + final Map 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 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 results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + List> results = coalesceAndAwait(futures); + + final List> 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>> 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 taskGroupsToVerify) { - final List> futures = new ArrayList<>(); + final List> futures = new ArrayList<>(); for (TaskGroup taskGroup : taskGroupsToVerify) { - futures.add(workerExec.submit(() -> verifyAndMergeCheckpoints(taskGroup))); + //noinspection unchecked + futures.add((ListenableFuture) 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>> futuresResult = - Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + List>>> futuresResult = + coalesceAndAwait(futures); + for (int i = 0; i < futuresResult.size(); i++) { - final TreeMap> checkpoints = futuresResult.get(i); + final Either>> 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 results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + final List> 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> results = Futures.successfulAsList(futures) - .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + List>> results = coalesceAndAwait(futures); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); TaskGroup group = activelyReadingTaskGroups.get(groupId); - Map endOffsets = results.get(j); - if (endOffsets != null) { + if (results.get(j).isValue() && results.get(j).valueOrThrow() != null) { + Map 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>, Map>() + FutureUtils.coalesce(pauseFutures), + new Function>>, Map>() { @Nullable @Override - public Map apply(List> input) + public Map apply(List>> input) { // 3) Build a map of the highest sequence read by any task in the group for each partition final Map endOffsets = new HashMap<>(); for (int i = 0; i < input.size(); i++) { - final Map 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 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 sequence : result.entrySet()) { + for (Entry 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 results = Futures.successfulAsList(setEndOffsetFutures) - .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + List> 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 stopTasksInGroup(@Nullable TaskGroup taskGroup, String stopReasonFormat, Object... args) + private ListenableFuture stopTasksInGroup( + @Nullable TaskGroup taskGroup, + String stopReasonFormat, + Object... args + ) { if (taskGroup == null) { return Futures.immediateFuture(null); @@ -3160,13 +3175,13 @@ public abstract class SeekableStreamSupervisor null); } private void checkPendingCompletionTasks() throws ExecutionException, InterruptedException, TimeoutException { - List> futures = new ArrayList<>(); + List> futures = new ArrayList<>(); for (Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { @@ -3262,15 +3277,15 @@ public abstract class SeekableStreamSupervisor activeTaskMap = getActiveTaskMap(); - List> futures = new ArrayList<>(); + List> futures = new ArrayList<>(); Iterator> iTaskGroups = activelyReadingTaskGroups.entrySet().iterator(); while (iTaskGroups.hasNext()) { Entry taskGroupEntry = iTaskGroups.next(); @@ -3320,8 +3335,8 @@ public abstract class SeekableStreamSupervisor List> coalesceAndAwait(final List> futures) + throws ExecutionException, InterruptedException, TimeoutException + { + final ListenableFuture>> 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 { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index ac985ac5b03..0ac57f8df72 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -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 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 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); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java index 733f1258ccb..c74c2952de0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -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(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java new file mode 100644 index 00000000000..0fa4e25ae32 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java @@ -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 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> 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 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 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 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 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 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 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 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 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 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 + { + public TestSeekableStreamIndexTaskClientAsyncImpl() + { + super(DATASOURCE, serviceClientFactory, null, jsonMapper, httpTimeout, MAX_ATTEMPTS); + } + + @Override + public Class getPartitionType() + { + return Integer.class; + } + + @Override + public Class getSequenceType() + { + return Long.class; + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java index 519eff9c09f..b449867546f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java @@ -447,6 +447,12 @@ public class SeekableStreamSupervisorSpecTest extends EasyMockSupport return 1; } + @Override + public boolean getChatAsync() + { + return false; + } + @Override public Integer getChatThreads() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 278b026c2da..ab3f9c5f9f1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -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> 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 filterMetrics(List events, List whitelist) { List result = events.stream() @@ -1093,6 +1141,12 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport return 1; } + @Override + public boolean getChatAsync() + { + return false; + } + @Override public Integer getChatThreads() { diff --git a/server/src/main/java/org/apache/druid/rpc/RequestBuilder.java b/server/src/main/java/org/apache/druid/rpc/RequestBuilder.java index bc75d848f5a..224cfc78ed1 100644 --- a/server/src/main/java/org/apache/druid/rpc/RequestBuilder.java +++ b/server/src/main/java/org/apache/druid/rpc/RequestBuilder.java @@ -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 + + '}'; + } } diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClient.java b/server/src/main/java/org/apache/druid/rpc/ServiceClient.java index 458016c867b..cb33f713d98 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceClient.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClient.java @@ -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 diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java b/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java index e7fbf72b1d3..1445b943e96 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java @@ -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, diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceRetryPolicy.java b/server/src/main/java/org/apache/druid/rpc/ServiceRetryPolicy.java index 786a7b1ab8e..297f369688a 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceRetryPolicy.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceRetryPolicy.java @@ -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(); } diff --git a/server/src/main/java/org/apache/druid/rpc/StandardRetryPolicy.java b/server/src/main/java/org/apache/druid/rpc/StandardRetryPolicy.java index 5cb4e74e47a..787bcae1e69 100644 --- a/server/src/main/java/org/apache/druid/rpc/StandardRetryPolicy.java +++ b/server/src/main/java/org/apache/druid/rpc/StandardRetryPolicy.java @@ -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); } } } diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java b/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java index fe88e582396..2482786c2a7 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java @@ -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. diff --git a/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java b/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java new file mode 100644 index 00000000000..60e6a347048 --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java @@ -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 expectations = new ArrayDeque<>(16); + + @Override + public ListenableFuture asyncRequest( + final RequestBuilder requestBuilder, + final HttpResponseHandler handler + ) + { + final Expectation expectation = expectations.poll(); + + Assert.assertEquals( + "request", + expectation == null ? null : expectation.request, + requestBuilder + ); + + if (expectation.response.isValue()) { + final ClientResponse 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 headers, + final byte[] content + ) + { + final HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, status); + for (Map.Entry 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 response; + + public Expectation(RequestBuilder request, Either response) + { + this.request = request; + this.response = response; + } + + @Override + public String toString() + { + return "Expectation{" + + "request=" + request + + ", response=" + response + + '}'; + } + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java index f998bf82d7b..48f922e6ab6 100644 --- a/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java @@ -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() {