From 2b330186e2a35ead88f3e1a5bfcd4de8853d081a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 5 Jul 2022 09:43:26 -0700 Subject: [PATCH] Mid-level service client and updated high-level clients. (#12696) * Mid-level service client and updated high-level clients. Our servers talk to each other over HTTP. We have a low-level HTTP client (HttpClient) that is super-asynchronous and super-customizable through its handlers. It's also proven to be quite robust: we use it for Broker -> Historical communication over the wide variety of query types and workloads we support. But the low-level client has no facilities for service location or retries, which means we have a variety of high-level clients that implement these in their own ways. Some high-level clients do a better job than others. This patch adds a mid-level ServiceClient that makes it easier for high-level clients to be built correctly and harmoniously, and migrates some of the high-level logic to use ServiceClients. Main changes: 1) Add ServiceClient org.apache.druid.rpc package. That package also contains supporting stuff like ServiceLocator and RetryPolicy interfaces, and a DiscoveryServiceLocator based on DruidNodeDiscoveryProvider. 2) Add high-level OverlordClient in org.apache.druid.rpc.indexing. 3) Indexing task client creator in TaskServiceClients. It uses SpecificTaskServiceLocator to find the tasks. This improves on ClientInfoTaskProvider by caching task locations for up to 30 seconds across calls, reducing load on the Overlord. 4) Rework ParallelIndexSupervisorTaskClient to use a ServiceClient instead of extending IndexTaskClient. 5) Rework RemoteTaskActionClient to use a ServiceClient instead of DruidLeaderClient. 6) Rework LocalIntermediaryDataManager, TaskMonitor, and ParallelIndexSupervisorTask. As a result, MiddleManager, Peon, and Overlord no longer need IndexingServiceClient (which internally used DruidLeaderClient). There are some concrete benefits over the prior logic, namely: - DruidLeaderClient does retries in its "go" method, but only retries exactly 5 times, does not sleep between retries, and does not retry retryable HTTP codes like 502, 503, 504. (It only retries IOExceptions.) ServiceClient handles retries in a more reasonable way. - DruidLeaderClient's methods are all synchronous, whereas ServiceClient methods are asynchronous. This is used in one place so far: the SpecificTaskServiceLocator, so we don't need to block a thread trying to locate a task. It can be used in other places in the future. - HttpIndexingServiceClient does not properly handle all server errors. In some cases, it tries to parse a server error as a successful response (for example: in getTaskStatus). - IndexTaskClient currently makes an Overlord call on every task-to-task HTTP request, as a way to find where the target task is. ServiceClient, through SpecificTaskServiceLocator, caches these target locations for a period of time. * Style adjustments. * For the coverage. * Adjustments. * Better behaviors. * Fixes. --- .../druid/common/guava/FutureUtils.java | 87 +++ .../druid/common/guava/FutureUtilsTest.java | 172 +++++ .../indexing/kafka/KafkaIndexTaskTest.java | 4 +- .../kinesis/KinesisIndexTaskTest.java | 4 +- .../indexing/common/IndexTaskClient.java | 5 - .../druid/indexing/common/TaskToolbox.java | 41 +- .../indexing/common/TaskToolboxFactory.java | 21 +- .../actions/RemoteTaskActionClient.java | 114 ++-- .../RemoteTaskActionClientFactory.java | 33 +- .../task/ClientBasedTaskInfoProvider.java | 55 -- ...visorTaskCoordinatingSegmentAllocator.java | 6 +- .../parallel/ParallelIndexPhaseRunner.java | 2 +- .../parallel/ParallelIndexSupervisorTask.java | 32 +- .../ParallelIndexSupervisorTaskClient.java | 104 +-- ...ParallelIndexSupervisorTaskClientImpl.java | 109 +++ ...lelIndexSupervisorTaskClientProvider.java} | 18 +- ...IndexSupervisorTaskClientProviderImpl.java | 73 ++ .../ParallelIndexTaskClientFactory.java | 66 -- .../PartialDimensionCardinalityTask.java | 16 +- .../PartialDimensionDistributionTask.java | 9 +- .../parallel/PartialSegmentGenerateTask.java | 9 +- .../parallel/PartialSegmentMergeTask.java | 12 +- .../batch/parallel/SinglePhaseSubTask.java | 9 +- .../task/batch/parallel/TaskMonitor.java | 34 +- .../shuffle/LocalIntermediaryDataManager.java | 17 +- .../indexing/common/TaskToolboxTest.java | 4 +- .../druid/indexing/common/TestUtils.java | 17 +- .../actions/RemoteTaskActionClientTest.java | 106 +-- ...penderatorDriverRealtimeIndexTaskTest.java | 4 +- .../ClientCompactionTaskQuerySerdeTest.java | 6 +- .../common/task/CompactionTaskRunTest.java | 10 +- .../common/task/CompactionTaskTest.java | 5 - .../common/task/IngestionTestBase.java | 2 - .../common/task/RealtimeIndexTaskTest.java | 4 +- ...bstractMultiPhaseParallelIndexingTest.java | 3 +- ...stractParallelIndexSupervisorTaskTest.java | 67 +- ...rallelIndexSupervisorTaskResourceTest.java | 10 +- .../ParallelIndexSupervisorTaskTest.java | 80 +++ .../parallel/ParallelIndexTestingFactory.java | 10 +- .../PartialDimensionCardinalityTaskTest.java | 26 +- .../PartialDimensionDistributionTaskTest.java | 26 +- .../task/batch/parallel/TaskMonitorTest.java | 17 +- .../SingleTaskBackgroundRunnerTest.java | 4 +- .../indexing/overlord/TaskLifecycleTest.java | 4 +- .../worker/WorkerTaskManagerTest.java | 4 +- .../worker/WorkerTaskMonitorTest.java | 4 +- ...ntermediaryDataManagerAutoCleanupTest.java | 16 +- ...iaryDataManagerManualAddAndDeleteTest.java | 8 +- .../shuffle/ShuffleDataSegmentPusherTest.java | 8 +- .../worker/shuffle/ShuffleResourceTest.java | 14 +- .../indexing/HttpIndexingServiceClient.java | 21 - .../indexing/IndexingServiceClient.java | 10 +- .../druid/initialization/Initialization.java | 4 +- .../druid/rpc/DiscoveryServiceLocator.java | 161 +++++ .../druid/rpc/HttpResponseException.java | 61 ++ .../druid/rpc/IgnoreHttpResponseHandler.java | 63 ++ .../org/apache/druid/rpc/RequestBuilder.java | 186 +++++ .../org/apache/druid/rpc/RpcException.java | 40 ++ .../org/apache/druid/rpc/ServiceClient.java | 97 +++ .../druid/rpc/ServiceClientFactory.java | 36 + .../druid/rpc/ServiceClientFactoryImpl.java | 52 ++ .../apache/druid/rpc/ServiceClientImpl.java | 424 ++++++++++++ .../druid/rpc/ServiceClosedException.java | 31 + .../org/apache/druid/rpc/ServiceLocation.java | 102 +++ .../apache/druid/rpc/ServiceLocations.java | 99 +++ .../org/apache/druid/rpc/ServiceLocator.java | 45 ++ .../rpc/ServiceNotAvailableException.java | 31 + .../apache/druid/rpc/ServiceRetryPolicy.java | 56 ++ .../apache/druid/rpc/StandardRetryPolicy.java | 141 ++++ .../druid/rpc/guice/ServiceClientModule.java | 99 +++ .../druid/rpc/indexing/OverlordClient.java | 56 ++ .../rpc/indexing/OverlordClientImpl.java | 159 +++++ .../rpc/indexing/SpecificTaskRetryPolicy.java | 91 +++ .../indexing/SpecificTaskServiceLocator.java | 198 ++++++ .../rpc/indexing/TaskServiceClients.java | 53 ++ .../indexing/NoopIndexingServiceClient.java | 138 ---- .../client/indexing/NoopOverlordClient.java | 67 ++ .../rpc/DiscoveryServiceLocatorTest.java | 201 ++++++ .../rpc/NoDelayScheduledExecutorService.java | 127 ++++ .../apache/druid/rpc/RequestBuilderTest.java | 173 +++++ .../druid/rpc/ServiceClientImplTest.java | 636 ++++++++++++++++++ .../apache/druid/rpc/ServiceLocationTest.java | 34 + .../druid/rpc/ServiceLocationsTest.java | 69 ++ .../SpecificTaskServiceLocatorTest.java | 172 +++++ .../apache/druid/cli/CliMiddleManager.java | 10 +- .../org/apache/druid/cli/CliOverlord.java | 10 +- .../java/org/apache/druid/cli/CliPeon.java | 13 +- 87 files changed, 4654 insertions(+), 823 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/common/guava/FutureUtils.java create mode 100644 core/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/ClientBasedTaskInfoProvider.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientImpl.java rename indexing-service/src/{test/java/org/apache/druid/indexing/common/task/NoopIndexTaskClientFactory.java => main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientProvider.java} (67%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientProviderImpl.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java create mode 100644 server/src/main/java/org/apache/druid/rpc/DiscoveryServiceLocator.java create mode 100644 server/src/main/java/org/apache/druid/rpc/HttpResponseException.java create mode 100644 server/src/main/java/org/apache/druid/rpc/IgnoreHttpResponseHandler.java create mode 100644 server/src/main/java/org/apache/druid/rpc/RequestBuilder.java create mode 100644 server/src/main/java/org/apache/druid/rpc/RpcException.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceClient.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceClientFactory.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceClientFactoryImpl.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceClosedException.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceLocation.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceLocations.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceLocator.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceNotAvailableException.java create mode 100644 server/src/main/java/org/apache/druid/rpc/ServiceRetryPolicy.java create mode 100644 server/src/main/java/org/apache/druid/rpc/StandardRetryPolicy.java create mode 100644 server/src/main/java/org/apache/druid/rpc/guice/ServiceClientModule.java create mode 100644 server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java create mode 100644 server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java create mode 100644 server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java create mode 100644 server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskServiceLocator.java create mode 100644 server/src/main/java/org/apache/druid/rpc/indexing/TaskServiceClients.java delete mode 100644 server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java create mode 100644 server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java create mode 100644 server/src/test/java/org/apache/druid/rpc/DiscoveryServiceLocatorTest.java create mode 100644 server/src/test/java/org/apache/druid/rpc/NoDelayScheduledExecutorService.java create mode 100644 server/src/test/java/org/apache/druid/rpc/RequestBuilderTest.java create mode 100644 server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java create mode 100644 server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java create mode 100644 server/src/test/java/org/apache/druid/rpc/ServiceLocationsTest.java create mode 100644 server/src/test/java/org/apache/druid/rpc/indexing/SpecificTaskServiceLocatorTest.java 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 new file mode 100644 index 00000000000..33b430ed7ba --- /dev/null +++ b/core/src/main/java/org/apache/druid/common/guava/FutureUtils.java @@ -0,0 +1,87 @@ +/* + * 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.common.guava; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; + +import java.util.concurrent.ExecutionException; +import java.util.function.Function; + +public class FutureUtils +{ + /** + * Waits for a given future and returns its value, like {@code future.get()}. + * + * On InterruptedException, cancels the provided future if {@code cancelIfInterrupted}, then re-throws the + * original InterruptedException. + * + * Passes through CancellationExceptions and ExecutionExceptions as-is. + */ + public static T get(final ListenableFuture future, final boolean cancelIfInterrupted) + throws InterruptedException, ExecutionException + { + try { + return future.get(); + } + catch (InterruptedException e) { + if (cancelIfInterrupted) { + future.cancel(true); + } + + throw e; + } + } + + /** + * Waits for a given future and returns its value, like {@code future.get()}. + * + * On InterruptException, cancels the provided future if {@code cancelIfInterrupted}, and in either case, throws + * a RuntimeException that wraps the original InterruptException. + * + * Passes through CancellationExceptions as-is. + * + * Re-wraps the causes of ExecutionExceptions using RuntimeException. + */ + public static T getUnchecked(final ListenableFuture future, final boolean cancelIfInterrupted) + { + try { + return FutureUtils.get(future, cancelIfInterrupted); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + catch (ExecutionException e) { + throw new RuntimeException(e.getCause()); + } + } + + /** + * Like {@link Futures#transform}, but works better with lambdas due to not having overloads. + * + * One can write {@code FutureUtils.transform(future, v -> ...)} instead of + * {@code Futures.transform(future, (Function) v -> ...)} + */ + public static ListenableFuture transform(final ListenableFuture future, final Function fn) + { + return Futures.transform(future, fn::apply); + } +} 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 new file mode 100644 index 00000000000..6f572ed1f19 --- /dev/null +++ b/core/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java @@ -0,0 +1,172 @@ +/* + * 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.common.guava; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.SettableFuture; +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.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +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.atomic.AtomicReference; + +public class FutureUtilsTest +{ + private ExecutorService exec; + + @Before + public void setUp() + { + exec = Execs.singleThreaded(StringUtils.encodeForFormat(getClass().getName()) + "-%d"); + } + + @After + public void tearDown() + { + if (exec != null) { + exec.shutdownNow(); + exec = null; + } + } + + @Test + public void test_get_ok() throws Exception + { + final String s = FutureUtils.get(Futures.immediateFuture("x"), true); + Assert.assertEquals("x", s); + } + + @Test + public void test_get_failed() + { + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> FutureUtils.get(Futures.immediateFailedFuture(new ISE("oh no")), true) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IllegalStateException.class)); + MatcherAssert.assertThat(e.getCause(), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("oh no"))); + } + + @Test + public void test_getUnchecked_interrupted_cancelOnInterrupt() throws InterruptedException + { + final SettableFuture neverGoingToResolve = SettableFuture.create(); + final AtomicReference exceptionFromOtherThread = new AtomicReference<>(); + final CountDownLatch runningLatch = new CountDownLatch(1); + + final Future execResult = exec.submit(() -> { + runningLatch.countDown(); + + try { + FutureUtils.getUnchecked(neverGoingToResolve, true); + } + catch (Throwable t) { + exceptionFromOtherThread.set(t); + } + }); + + runningLatch.await(); + Assert.assertTrue(execResult.cancel(true)); + exec.shutdown(); + + Assert.assertTrue(exec.awaitTermination(1, TimeUnit.MINUTES)); + exec = null; + + Assert.assertTrue(neverGoingToResolve.isCancelled()); + + final Throwable e = exceptionFromOtherThread.get(); + MatcherAssert.assertThat(e, CoreMatchers.instanceOf(RuntimeException.class)); + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(InterruptedException.class)); + } + + @Test + public void test_getUnchecked_interrupted_dontCancelOnInterrupt() throws InterruptedException + { + final SettableFuture neverGoingToResolve = SettableFuture.create(); + final AtomicReference exceptionFromOtherThread = new AtomicReference<>(); + final CountDownLatch runningLatch = new CountDownLatch(1); + + final Future execResult = exec.submit(() -> { + runningLatch.countDown(); + + try { + FutureUtils.getUnchecked(neverGoingToResolve, false); + } + catch (Throwable t) { + exceptionFromOtherThread.set(t); + } + }); + + runningLatch.await(); + Assert.assertTrue(execResult.cancel(true)); + exec.shutdown(); + + Assert.assertTrue(exec.awaitTermination(1, TimeUnit.MINUTES)); + exec = null; + + Assert.assertFalse(neverGoingToResolve.isCancelled()); + Assert.assertFalse(neverGoingToResolve.isDone()); + + final Throwable e = exceptionFromOtherThread.get(); + MatcherAssert.assertThat(e, CoreMatchers.instanceOf(RuntimeException.class)); + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(InterruptedException.class)); + } + + @Test + public void test_getUnchecked_ok() + { + final String s = FutureUtils.getUnchecked(Futures.immediateFuture("x"), true); + Assert.assertEquals("x", s); + } + + @Test + public void test_getUnchecked_failed() + { + final RuntimeException e = Assert.assertThrows( + RuntimeException.class, + () -> FutureUtils.getUnchecked(Futures.immediateFailedFuture(new ISE("oh no")), true) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IllegalStateException.class)); + MatcherAssert.assertThat(e.getCause(), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("oh no"))); + } + + @Test + public void test_transform() throws Exception + { + Assert.assertEquals( + "xy", + FutureUtils.transform(Futures.immediateFuture("x"), s -> s + "y").get() + ); + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index afe9c68efe4..6e7bee5b240 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -39,7 +39,7 @@ import org.apache.curator.test.TestingCluster; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputFormat; @@ -3147,7 +3147,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase new NoopChatHandlerProvider(), testUtils.getRowIngestionMetersFactory(), new TestAppenderatorsManager(), - new NoopIndexingServiceClient(), + new NoopOverlordClient(), null, null, null diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index e48c5a58726..672dae3fd9a 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -38,7 +38,7 @@ import com.google.inject.name.Named; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -3158,7 +3158,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase new NoopChatHandlerProvider(), testUtils.getRowIngestionMetersFactory(), new TestAppenderatorsManager(), - new NoopIndexingServiceClient(), + new NoopOverlordClient(), null, null, null 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 db1692a3e09..21de9f21244 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 @@ -140,11 +140,6 @@ public abstract class IndexTaskClient implements AutoCloseable ); } - protected HttpClient getHttpClient() - { - return httpClient; - } - protected RetryPolicy newRetryPolicy() { return retryPolicyFactory.makeRetryPolicy(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index fae02fe08e0..551a4d1dcde 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -31,15 +31,13 @@ import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.indexing.common.actions.SegmentInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.worker.shuffle.IntermediaryDataManager; import org.apache.druid.java.util.common.FileUtils; @@ -48,6 +46,7 @@ import org.apache.druid.java.util.metrics.Monitor; import org.apache.druid.java.util.metrics.MonitorScheduler; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; @@ -120,12 +119,12 @@ public class TaskToolbox private final ChatHandlerProvider chatHandlerProvider; private final RowIngestionMetersFactory rowIngestionMetersFactory; private final AppenderatorsManager appenderatorsManager; - private final IndexingServiceClient indexingServiceClient; + private final OverlordClient overlordClient; private final CoordinatorClient coordinatorClient; // Used by only native parallel tasks private final IntermediaryDataManager intermediaryDataManager; - private final IndexTaskClientFactory supervisorTaskClientFactory; + private final ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider; private final ShuffleClient shuffleClient; public TaskToolbox( @@ -162,9 +161,9 @@ public class TaskToolbox ChatHandlerProvider chatHandlerProvider, RowIngestionMetersFactory rowIngestionMetersFactory, AppenderatorsManager appenderatorsManager, - IndexingServiceClient indexingServiceClient, + OverlordClient overlordClient, CoordinatorClient coordinatorClient, - IndexTaskClientFactory supervisorTaskClientFactory, + ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider, ShuffleClient shuffleClient ) { @@ -202,9 +201,9 @@ public class TaskToolbox this.chatHandlerProvider = chatHandlerProvider; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.appenderatorsManager = appenderatorsManager; - this.indexingServiceClient = indexingServiceClient; + this.overlordClient = overlordClient; this.coordinatorClient = coordinatorClient; - this.supervisorTaskClientFactory = supervisorTaskClientFactory; + this.supervisorTaskClientProvider = supervisorTaskClientProvider; this.shuffleClient = shuffleClient; } @@ -442,9 +441,9 @@ public class TaskToolbox return appenderatorsManager; } - public IndexingServiceClient getIndexingServiceClient() + public OverlordClient getOverlordClient() { - return indexingServiceClient; + return overlordClient; } public CoordinatorClient getCoordinatorClient() @@ -452,9 +451,9 @@ public class TaskToolbox return coordinatorClient; } - public IndexTaskClientFactory getSupervisorTaskClientFactory() + public ParallelIndexSupervisorTaskClientProvider getSupervisorTaskClientProvider() { - return supervisorTaskClientFactory; + return supervisorTaskClientProvider; } public ShuffleClient getShuffleClient() @@ -496,10 +495,10 @@ public class TaskToolbox private ChatHandlerProvider chatHandlerProvider; private RowIngestionMetersFactory rowIngestionMetersFactory; private AppenderatorsManager appenderatorsManager; - private IndexingServiceClient indexingServiceClient; + private OverlordClient overlordClient; private CoordinatorClient coordinatorClient; private IntermediaryDataManager intermediaryDataManager; - private IndexTaskClientFactory supervisorTaskClientFactory; + private ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider; private ShuffleClient shuffleClient; public Builder() @@ -698,9 +697,9 @@ public class TaskToolbox return this; } - public Builder indexingServiceClient(final IndexingServiceClient indexingServiceClient) + public Builder overlordClient(final OverlordClient overlordClient) { - this.indexingServiceClient = indexingServiceClient; + this.overlordClient = overlordClient; return this; } @@ -716,9 +715,9 @@ public class TaskToolbox return this; } - public Builder supervisorTaskClientFactory(final IndexTaskClientFactory supervisorTaskClientFactory) + public Builder supervisorTaskClientProvider(final ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider) { - this.supervisorTaskClientFactory = supervisorTaskClientFactory; + this.supervisorTaskClientProvider = supervisorTaskClientProvider; return this; } @@ -764,9 +763,9 @@ public class TaskToolbox chatHandlerProvider, rowIngestionMetersFactory, appenderatorsManager, - indexingServiceClient, + overlordClient, coordinatorClient, - supervisorTaskClientFactory, + supervisorTaskClientProvider, shuffleClient ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 9ccc94659b0..7c0b8efee40 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -27,7 +27,6 @@ import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.LookupNodeService; @@ -36,16 +35,16 @@ import org.apache.druid.guice.annotations.Parent; import org.apache.druid.guice.annotations.RemoteChatHandler; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.worker.shuffle.IntermediaryDataManager; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.MonitorScheduler; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; @@ -100,12 +99,12 @@ public class TaskToolboxFactory private final ChatHandlerProvider chatHandlerProvider; private final RowIngestionMetersFactory rowIngestionMetersFactory; private final AppenderatorsManager appenderatorsManager; - private final IndexingServiceClient indexingServiceClient; + private final OverlordClient overlordClient; private final CoordinatorClient coordinatorClient; // Used by only native parallel tasks private final IntermediaryDataManager intermediaryDataManager; - private final IndexTaskClientFactory supervisorTaskClientFactory; + private final ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider; private final ShuffleClient shuffleClient; @Inject @@ -142,9 +141,9 @@ public class TaskToolboxFactory ChatHandlerProvider chatHandlerProvider, RowIngestionMetersFactory rowIngestionMetersFactory, AppenderatorsManager appenderatorsManager, - IndexingServiceClient indexingServiceClient, + OverlordClient overlordClient, CoordinatorClient coordinatorClient, - IndexTaskClientFactory supervisorTaskClientFactory, + ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider, ShuffleClient shuffleClient ) { @@ -180,9 +179,9 @@ public class TaskToolboxFactory this.chatHandlerProvider = chatHandlerProvider; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.appenderatorsManager = appenderatorsManager; - this.indexingServiceClient = indexingServiceClient; + this.overlordClient = overlordClient; this.coordinatorClient = coordinatorClient; - this.supervisorTaskClientFactory = supervisorTaskClientFactory; + this.supervisorTaskClientProvider = supervisorTaskClientProvider; this.shuffleClient = shuffleClient; } @@ -227,9 +226,9 @@ public class TaskToolboxFactory .chatHandlerProvider(chatHandlerProvider) .rowIngestionMetersFactory(rowIngestionMetersFactory) .appenderatorsManager(appenderatorsManager) - .indexingServiceClient(indexingServiceClient) + .overlordClient(overlordClient) .coordinatorClient(coordinatorClient) - .supervisorTaskClientFactory(supervisorTaskClientFactory) + .supervisorTaskClientProvider(supervisorTaskClientProvider) .shuffleClient(shuffleClient) .build(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java index eb1d62da8df..4015b99558a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java @@ -20,43 +20,38 @@ package org.apache.druid.indexing.common.actions; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.discovery.DruidLeaderClient; -import org.apache.druid.indexing.common.RetryPolicy; -import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; -import org.jboss.netty.channel.ChannelException; +import org.apache.druid.rpc.HttpResponseException; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; import org.jboss.netty.handler.codec.http.HttpMethod; -import org.joda.time.Duration; -import javax.ws.rs.core.MediaType; import java.io.IOException; import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.ExecutionException; public class RemoteTaskActionClient implements TaskActionClient { private final Task task; - private final RetryPolicyFactory retryPolicyFactory; private final ObjectMapper jsonMapper; - private final DruidLeaderClient druidLeaderClient; + private final ServiceClient client; private static final Logger log = new Logger(RemoteTaskActionClient.class); public RemoteTaskActionClient( Task task, - DruidLeaderClient druidLeaderClient, - RetryPolicyFactory retryPolicyFactory, + ServiceClient client, ObjectMapper jsonMapper ) { this.task = task; - this.retryPolicyFactory = retryPolicyFactory; + this.client = client; this.jsonMapper = jsonMapper; - this.druidLeaderClient = druidLeaderClient; } @Override @@ -64,73 +59,40 @@ public class RemoteTaskActionClient implements TaskActionClient { log.debug("Performing action for task[%s]: %s", task.getId(), taskAction); - byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction)); + try { + // We're using a ServiceClient directly here instead of OverlordClient, because OverlordClient does + // not have access to the TaskAction class. (OverlordClient is in the druid-server package, and TaskAction + // is in the druid-indexing-service package.) + final Map response = jsonMapper.readValue( + client.request( + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/action") + .jsonContent(jsonMapper, new TaskActionHolder(task, taskAction)), + new BytesFullResponseHandler() + ).getContent(), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); - final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); - - while (true) { - try { - - final StringFullResponseHolder fullResponseHolder; - - log.debug( - "Submitting action for task[%s] to Overlord: %s", - task.getId(), - jsonMapper.writeValueAsString(taskAction) + return jsonMapper.convertValue( + response.get("result"), + taskAction.getReturnTypeReference() + ); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + catch (ExecutionException e) { + if (e.getCause() instanceof HttpResponseException) { + // Rewrite the error to be slightly more useful: point out that there may be information in the Overlord logs. + final StringFullResponseHolder fullResponseHolder = ((HttpResponseException) e.getCause()).getResponse(); + throw new IOE( + "Error with status[%s] and message[%s]. Check overlord logs for details.", + fullResponseHolder.getStatus(), + fullResponseHolder.getContent() ); - - fullResponseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/action") - .setContent(MediaType.APPLICATION_JSON, dataToSend) - ); - - if (fullResponseHolder.getStatus().getCode() / 100 == 2) { - final Map responseDict = jsonMapper.readValue( - fullResponseHolder.getContent(), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ); - return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference()); - } else { - // Want to retry, so throw an IOException. - throw new IOE( - "Error with status[%s] and message[%s]. Check overlord logs for details.", - fullResponseHolder.getStatus(), - fullResponseHolder.getContent() - ); - } } - catch (IOException | ChannelException e) { - log.noStackTrace().warn( - e, - "Exception submitting action for task[%s]: %s", - task.getId(), - jsonMapper.writeValueAsString(taskAction) - ); - final Duration delay = retryPolicy.getAndIncrementRetryDelay(); - if (delay == null) { - throw e; - } else { - try { - final long sleepTime = jitter(delay.getMillis()); - log.warn("Will try again in [%s].", new Duration(sleepTime).toString()); - Thread.sleep(sleepTime); - } - catch (InterruptedException e2) { - throw new RuntimeException(e2); - } - } - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } + throw new IOException(e.getCause()); } } - - private long jitter(long input) - { - final double jitter = ThreadLocalRandom.current().nextGaussian() * input / 4.0; - long retval = input + (long) jitter; - return retval < 0 ? 0 : retval; - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientFactory.java index f4b3ad20906..37bf503118e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientFactory.java @@ -22,33 +22,46 @@ package org.apache.druid.indexing.common.actions; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import org.apache.druid.client.indexing.IndexingService; -import org.apache.druid.discovery.DruidLeaderClient; -import org.apache.druid.indexing.common.RetryPolicyFactory; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocator; +import org.apache.druid.rpc.StandardRetryPolicy; /** */ public class RemoteTaskActionClientFactory implements TaskActionClientFactory { - private final DruidLeaderClient druidLeaderClient; - private final RetryPolicyFactory retryPolicyFactory; + private final ServiceClient overlordClient; private final ObjectMapper jsonMapper; @Inject public RemoteTaskActionClientFactory( - @IndexingService DruidLeaderClient leaderHttpClient, - RetryPolicyFactory retryPolicyFactory, - ObjectMapper jsonMapper + @Json final ObjectMapper jsonMapper, + @EscalatedGlobal final ServiceClientFactory clientFactory, + @IndexingService final ServiceLocator serviceLocator, + RetryPolicyConfig retryPolicyConfig ) { - this.druidLeaderClient = leaderHttpClient; - this.retryPolicyFactory = retryPolicyFactory; + this.overlordClient = clientFactory.makeClient( + NodeRole.OVERLORD.toString(), + serviceLocator, + StandardRetryPolicy.builder() + .maxAttempts(retryPolicyConfig.getMaxRetryCount() - 1) + .minWaitMillis(retryPolicyConfig.getMinWait().toStandardDuration().getMillis()) + .maxWaitMillis(retryPolicyConfig.getMaxWait().toStandardDuration().getMillis()) + .build() + ); this.jsonMapper = jsonMapper; } @Override public TaskActionClient create(Task task) { - return new RemoteTaskActionClient(task, druidLeaderClient, retryPolicyFactory, jsonMapper); + return new RemoteTaskActionClient(task, overlordClient, jsonMapper); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ClientBasedTaskInfoProvider.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ClientBasedTaskInfoProvider.java deleted file mode 100644 index cc480bcf9c0..00000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ClientBasedTaskInfoProvider.java +++ /dev/null @@ -1,55 +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.common.task; - -import com.google.common.base.Optional; -import com.google.inject.Inject; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.client.indexing.TaskStatusResponse; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.TaskInfoProvider; - -public class ClientBasedTaskInfoProvider implements TaskInfoProvider -{ - private final IndexingServiceClient client; - - @Inject - public ClientBasedTaskInfoProvider(IndexingServiceClient client) - { - this.client = client; - } - - @Override - public TaskLocation getTaskLocation(String id) - { - final TaskStatusResponse response = client.getTaskStatus(id); - return response == null ? TaskLocation.unknown() : response.getStatus().getLocation(); - } - - @Override - public Optional getTaskStatus(String id) - { - final TaskStatusResponse response = client.getTaskStatus(id); - return response == null ? - Optional.absent() : - Optional.of(TaskStatus.fromCode(id, response.getStatus().getStatusCode())); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java index bc72bfa1ec5..1c0e215cbe8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java @@ -31,7 +31,6 @@ import java.io.IOException; */ public class SupervisorTaskCoordinatingSegmentAllocator implements SegmentAllocatorForBatch { - private final String supervisorTaskId; private final ParallelIndexSupervisorTaskClient taskClient; private final SequenceNameFunction sequenceNameFunction; private final boolean useLineageBasedSegmentAllocation; @@ -42,7 +41,6 @@ public class SupervisorTaskCoordinatingSegmentAllocator implements SegmentAlloca boolean useLineageBasedSegmentAllocation ) { - this.supervisorTaskId = supervisorTaskAccess.getSupervisorTaskId(); this.taskClient = supervisorTaskAccess.getTaskClient(); this.sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(taskId); this.useLineageBasedSegmentAllocation = useLineageBasedSegmentAllocation; @@ -57,9 +55,9 @@ public class SupervisorTaskCoordinatingSegmentAllocator implements SegmentAlloca ) throws IOException { if (useLineageBasedSegmentAllocation) { - return taskClient.allocateSegment(supervisorTaskId, row.getTimestamp(), sequenceName, previousSegmentId); + return taskClient.allocateSegment(row.getTimestamp(), sequenceName, previousSegmentId); } else { - return taskClient.allocateSegment(supervisorTaskId, row.getTimestamp()); + return taskClient.allocateSegment(row.getTimestamp()); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java index 672fd7b5ae4..4725b43d64f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java @@ -120,7 +120,7 @@ public abstract class ParallelIndexPhaseRunner( - toolbox.getIndexingServiceClient(), + toolbox.getOverlordClient(), tuningConfig.getMaxRetry(), estimateTotalNumSubTasks() ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 058b869ddaa..f8381ffafe8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -29,6 +29,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.hll.Union; import org.apache.datasketches.memory.Memory; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; @@ -61,6 +62,8 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.rpc.HttpResponseException; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.incremental.MutableRowIngestionMeters; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -80,6 +83,7 @@ import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; import org.apache.druid.utils.CollectionUtils; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -110,6 +114,7 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; @@ -1338,7 +1343,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen /** * Worker tasks spawned by the supervisor call this API to report the segments they generated and pushed. * - * @see ParallelIndexSupervisorTaskClient#report(String, SubTaskReport) + * @see ParallelIndexSupervisorTaskClient#report(SubTaskReport) */ @POST @Path("/report") @@ -1623,7 +1628,8 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen final MutableRowIngestionMeters buildSegmentsRowStats = new MutableRowIngestionMeters(); for (String runningTaskId : runningTaskIds) { try { - Map report = toolbox.getIndexingServiceClient().getTaskReport(runningTaskId); + final Map report = getTaskReport(toolbox.getOverlordClient(), runningTaskId); + if (report == null || report.isEmpty()) { // task does not have a running report yet continue; @@ -1774,6 +1780,28 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen return Response.ok(doGetLiveReports(full)).build(); } + /** + * Like {@link OverlordClient#taskReportAsMap}, but synchronous, and returns null instead of throwing an error if + * the server returns 404. + */ + @Nullable + @VisibleForTesting + static Map getTaskReport(final OverlordClient overlordClient, final String taskId) + throws InterruptedException, ExecutionException + { + try { + return FutureUtils.get(overlordClient.taskReportAsMap(taskId), true); + } + catch (ExecutionException e) { + if (e.getCause() instanceof HttpResponseException && + ((HttpResponseException) e.getCause()).getResponse().getStatus().equals(HttpResponseStatus.NOT_FOUND)) { + return null; + } else { + throw e; + } + } + } + /** * Represents a partition uniquely identified by an Interval and a bucketId. * diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClient.java index f12342eaf9a..ac2cdebcf49 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClient.java @@ -19,122 +19,28 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.indexing.common.IndexTaskClient; -import org.apache.druid.indexing.common.TaskInfoProvider; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.DateTime; -import org.joda.time.Duration; import javax.annotation.Nullable; import java.io.IOException; -public class ParallelIndexSupervisorTaskClient extends IndexTaskClient +public interface ParallelIndexSupervisorTaskClient { - ParallelIndexSupervisorTaskClient( - HttpClient httpClient, - ObjectMapper objectMapper, - TaskInfoProvider taskInfoProvider, - Duration httpTimeout, - String callerId, - long numRetries - ) - { - super(httpClient, objectMapper, taskInfoProvider, httpTimeout, callerId, 1, numRetries); - } - /** * See {@link SinglePhaseParallelIndexTaskRunner#allocateNewSegment(String, DateTime)}. */ @Deprecated - public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException - { - final StringFullResponseHolder response = submitSmileRequest( - supervisorTaskId, - HttpMethod.POST, - "segment/allocate", - null, - serialize(timestamp), - true - ); - if (!isSuccess(response)) { - throw new ISE( - "task[%s] failed to allocate a new segment identifier with the HTTP code[%d] and content[%s]", - supervisorTaskId, - response.getStatus().getCode(), - response.getContent() - ); - } else { - return deserialize( - response.getContent(), - new TypeReference() - { - } - ); - } - } + SegmentIdWithShardSpec allocateSegment(DateTime timestamp) throws IOException; /** * See {@link SinglePhaseParallelIndexTaskRunner#allocateNewSegment(String, DateTime, String, String)}. */ - public SegmentIdWithShardSpec allocateSegment( - String supervisorTaskId, + SegmentIdWithShardSpec allocateSegment( DateTime timestamp, String sequenceName, @Nullable String prevSegmentId - ) throws IOException - { - final StringFullResponseHolder response = submitSmileRequest( - supervisorTaskId, - HttpMethod.POST, - "segment/allocate", - null, - serialize(new SegmentAllocationRequest(timestamp, sequenceName, prevSegmentId)), - true - ); - if (!isSuccess(response)) { - throw new ISE( - "task[%s] failed to allocate a new segment identifier with the HTTP code[%d] and content[%s]", - supervisorTaskId, - response.getStatus().getCode(), - response.getContent() - ); - } else { - return deserialize( - response.getContent(), - new TypeReference() - { - } - ); - } - } + ) throws IOException; - public void report(String supervisorTaskId, SubTaskReport report) - { - try { - final StringFullResponseHolder response = submitSmileRequest( - supervisorTaskId, - HttpMethod.POST, - "report", - null, - serialize(report), - true - ); - if (!isSuccess(response)) { - throw new ISE( - "Failed to send taskReports to task[%s] with the HTTP code [%d]", - supervisorTaskId, - response.getStatus().getCode() - ); - } - } - catch (IOException e) { - throw new RuntimeException(e); - } - } + void report(SubTaskReport report); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientImpl.java new file mode 100644 index 00000000000..ef6adff2fe2 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientImpl.java @@ -0,0 +1,109 @@ +/* + * 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.common.task.batch.parallel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.rpc.IgnoreHttpResponseHandler; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.joda.time.DateTime; +import org.joda.time.Duration; + +import javax.annotation.Nullable; +import java.io.IOException; + +public class ParallelIndexSupervisorTaskClientImpl implements ParallelIndexSupervisorTaskClient +{ + private final ServiceClient client; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + private final Duration httpTimeout; + + public ParallelIndexSupervisorTaskClientImpl( + ServiceClient client, + ObjectMapper jsonMapper, + ObjectMapper smileMapper, + Duration httpTimeout + ) + { + this.client = client; + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + this.httpTimeout = httpTimeout; + } + + @Override + public SegmentIdWithShardSpec allocateSegment(DateTime timestamp) throws IOException + { + // API accepts Smile requests and sends JSON responses. + return jsonMapper.readValue( + FutureUtils.getUnchecked( + client.asyncRequest( + new RequestBuilder(HttpMethod.POST, "/segment/allocate") + .smileContent(smileMapper, timestamp) + .timeout(httpTimeout), + new BytesFullResponseHandler() + ), + true + ).getContent(), + SegmentIdWithShardSpec.class + ); + } + + @Override + public SegmentIdWithShardSpec allocateSegment( + DateTime timestamp, + String sequenceName, + @Nullable String prevSegmentId + ) throws IOException + { + // API accepts Smile requests and sends JSON responses. + return jsonMapper.readValue( + FutureUtils.getUnchecked( + client.asyncRequest( + new RequestBuilder(HttpMethod.POST, "/segment/allocate") + .smileContent(smileMapper, new SegmentAllocationRequest(timestamp, sequenceName, prevSegmentId)) + .timeout(httpTimeout), + new BytesFullResponseHandler() + ), + true + ).getContent(), + SegmentIdWithShardSpec.class + ); + } + + @Override + public void report(SubTaskReport report) + { + FutureUtils.getUnchecked( + client.asyncRequest( + new RequestBuilder(HttpMethod.POST, "/report") + .smileContent(smileMapper, report) + .timeout(httpTimeout), + IgnoreHttpResponseHandler.INSTANCE + ), + true + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopIndexTaskClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientProvider.java similarity index 67% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopIndexTaskClientFactory.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientProvider.java index 8296d21a963..d594fb25e7a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopIndexTaskClientFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientProvider.java @@ -17,23 +17,15 @@ * under the License. */ -package org.apache.druid.indexing.common.task; +package org.apache.druid.indexing.common.task.batch.parallel; -import org.apache.druid.indexing.common.IndexTaskClient; -import org.apache.druid.indexing.common.TaskInfoProvider; import org.joda.time.Duration; -public class NoopIndexTaskClientFactory implements IndexTaskClientFactory +public interface ParallelIndexSupervisorTaskClientProvider { - @Override - public T build( - TaskInfoProvider taskInfoProvider, - String callerId, - int numThreads, + ParallelIndexSupervisorTaskClient build( + String supervisorTaskId, Duration httpTimeout, long numRetries - ) - { - throw new UnsupportedOperationException(); - } + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientProviderImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientProviderImpl.java new file mode 100644 index 00000000000..e191ff48528 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClientProviderImpl.java @@ -0,0 +1,73 @@ +/* + * 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.common.task.batch.parallel; + +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.guice.annotations.Smile; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.rpc.indexing.TaskServiceClients; +import org.joda.time.Duration; + +public class ParallelIndexSupervisorTaskClientProviderImpl implements ParallelIndexSupervisorTaskClientProvider +{ + private final ServiceClientFactory serviceClientFactory; + private final OverlordClient overlordClient; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + + @Inject + public ParallelIndexSupervisorTaskClientProviderImpl( + @EscalatedGlobal ServiceClientFactory serviceClientFactory, + OverlordClient overlordClient, + @Json ObjectMapper jsonMapper, + @Smile ObjectMapper smileMapper + ) + { + this.serviceClientFactory = serviceClientFactory; + this.overlordClient = overlordClient; + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + } + + @Override + public ParallelIndexSupervisorTaskClient build( + final String supervisorTaskId, + final Duration httpTimeout, + final long numRetries + ) + { + return new ParallelIndexSupervisorTaskClientImpl( + TaskServiceClients.makeClient( + supervisorTaskId, + StandardRetryPolicy.builder().maxAttempts(numRetries - 1).build(), + serviceClientFactory, + overlordClient + ), + jsonMapper, + smileMapper, + httpTimeout + ); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java deleted file mode 100644 index 73c44d37f36..00000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java +++ /dev/null @@ -1,66 +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.common.task.batch.parallel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.inject.Inject; -import org.apache.druid.guice.annotations.EscalatedGlobal; -import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.indexing.common.TaskInfoProvider; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.java.util.http.client.HttpClient; -import org.joda.time.Duration; - -public class ParallelIndexTaskClientFactory implements IndexTaskClientFactory -{ - private final HttpClient httpClient; - private final ObjectMapper mapper; - - @Inject - public ParallelIndexTaskClientFactory( - @EscalatedGlobal HttpClient httpClient, - @Smile ObjectMapper mapper - ) - { - this.httpClient = httpClient; - this.mapper = mapper; - } - - @Override - public ParallelIndexSupervisorTaskClient build( - TaskInfoProvider taskInfoProvider, - String callerId, - int numThreads, - Duration httpTimeout, - long numRetries - ) - { - Preconditions.checkState(numThreads == 1, "expect numThreads to be 1"); - return new ParallelIndexSupervisorTaskClient( - httpClient, - mapper, - taskInfoProvider, - httpTimeout, - callerId, - numRetries - ); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java index 394e1872f59..0b4def915ac 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java @@ -36,7 +36,6 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; -import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -250,13 +249,12 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask private void sendReport(TaskToolbox toolbox, DimensionCardinalityReport report) { - final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientFactory().build( - new ClientBasedTaskInfoProvider(toolbox.getIndexingServiceClient()), - getId(), - 1, // always use a single http thread - ingestionSchema.getTuningConfig().getChatHandlerTimeout(), - ingestionSchema.getTuningConfig().getChatHandlerNumRetries() - ); - taskClient.report(supervisorTaskId, report); + final ParallelIndexSupervisorTaskClient taskClient = + toolbox.getSupervisorTaskClientProvider().build( + supervisorTaskId, + ingestionSchema.getTuningConfig().getChatHandlerTimeout(), + ingestionSchema.getTuningConfig().getChatHandlerNumRetries() + ); + taskClient.report(report); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java index f83004b2035..e491e024425 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java @@ -39,7 +39,6 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; -import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistribution; import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch; @@ -303,14 +302,12 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask private void sendReport(TaskToolbox toolbox, DimensionDistributionReport report) { - final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientFactory().build( - new ClientBasedTaskInfoProvider(toolbox.getIndexingServiceClient()), - getId(), - 1, // always use a single http thread + final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( + supervisorTaskId, ingestionSchema.getTuningConfig().getChatHandlerTimeout(), ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); - taskClient.report(supervisorTaskId, report); + taskClient.report(report); } private interface InputRowFilter diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index b8704457226..449fec9f999 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -30,7 +30,6 @@ import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.BatchAppenderators; -import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.InputSourceProcessor; import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; @@ -114,10 +113,8 @@ abstract class PartialSegmentGenerateTask e ingestionSchema.getDataSchema().getParser() ); - final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientFactory().build( - new ClientBasedTaskInfoProvider(toolbox.getIndexingServiceClient()), - getId(), - 1, // always use a single http thread + final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( + supervisorTaskId, ingestionSchema.getTuningConfig().getChatHandlerTimeout(), ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); @@ -131,7 +128,7 @@ abstract class PartialSegmentGenerateTask e Map taskReport = getTaskCompletionReports(); - taskClient.report(supervisorTaskId, createGeneratedPartitionsReport(toolbox, segments, taskReport)); + taskClient.report(createGeneratedPartitionsReport(toolbox, segments, taskReport)); return TaskStatus.success(getId()); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index 0b2ba6e323f..7969b589fa8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -33,7 +33,6 @@ import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; -import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; @@ -179,10 +178,8 @@ abstract class PartialSegmentMergeTask extends PerfectRollu fetchStopwatch.stop(); LOG.info("Fetch took [%s] seconds", fetchTime); - final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientFactory().build( - new ClientBasedTaskInfoProvider(toolbox.getIndexingServiceClient()), - getId(), - 1, // always use a single http thread + final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( + supervisorTaskId, getTuningConfig().getChatHandlerTimeout(), getTuningConfig().getChatHandlerNumRetries() ); @@ -200,10 +197,7 @@ abstract class PartialSegmentMergeTask extends PerfectRollu intervalToUnzippedFiles ); - taskClient.report( - supervisorTaskId, - new PushedSegmentsReport(getId(), Collections.emptySet(), pushedSegments, ImmutableMap.of()) - ); + taskClient.report(new PushedSegmentsReport(getId(), Collections.emptySet(), pushedSegments, ImmutableMap.of())); return TaskStatus.success(getId()); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 0d47ea02d07..755a6baf93a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -41,7 +41,6 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.BatchAppenderators; -import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; @@ -252,10 +251,8 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand ingestionSchema.getDataSchema().getParser() ); - final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientFactory().build( - new ClientBasedTaskInfoProvider(toolbox.getIndexingServiceClient()), - getId(), - 1, // always use a single http thread + final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( + supervisorTaskId, ingestionSchema.getTuningConfig().getChatHandlerTimeout(), ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); @@ -277,7 +274,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand .toSet(); Map taskReport = getTaskCompletionReports(); - taskClient.report(supervisorTaskId, new PushedSegmentsReport(getId(), oldSegments, pushedSegments, taskReport)); + taskClient.report(new PushedSegmentsReport(getId(), oldSegments, pushedSegments, taskReport)); toolbox.getTaskReportFileWriter().write(getId(), taskReport); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java index 5f00c28d46d..8db78a10b58 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java @@ -25,14 +25,16 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.errorprone.annotations.concurrent.GuardedBy; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; import javax.annotation.Nullable; import java.util.Iterator; @@ -83,7 +85,7 @@ public class TaskMonitor> taskFuture = SettableFuture.create(); + final TaskStatusResponse statusResponse = FutureUtils.getUnchecked(overlordClient.taskStatus(task.getId()), true); runningTasks.put( spec.getId(), - new MonitorEntry(spec, task, indexingServiceClient.getTaskStatus(task.getId()).getStatus(), taskFuture) + new MonitorEntry(spec, task, statusResponse.getStatus(), taskFuture) ); return taskFuture; @@ -295,13 +303,11 @@ public class TaskMonitor shuffleDataLocations; - private final IndexingServiceClient indexingServiceClient; + private final OverlordClient overlordClient; // supervisorTaskId -> time to check supervisorTask status // This time is initialized when a new supervisorTask is found and updated whenever a partition is accessed for @@ -112,7 +113,7 @@ public class LocalIntermediaryDataManager implements IntermediaryDataManager public LocalIntermediaryDataManager( WorkerConfig workerConfig, TaskConfig taskConfig, - IndexingServiceClient indexingServiceClient + OverlordClient overlordClient ) { this.intermediaryPartitionDiscoveryPeriodSec = workerConfig.getIntermediaryPartitionDiscoveryPeriodSec(); @@ -124,7 +125,7 @@ public class LocalIntermediaryDataManager implements IntermediaryDataManager .stream() .map(config -> new StorageLocation(config.getPath(), config.getMaxSize(), config.getFreeSpacePercent())) .collect(Collectors.toList()); - this.indexingServiceClient = indexingServiceClient; + this.overlordClient = overlordClient; } @Override @@ -153,9 +154,6 @@ public class LocalIntermediaryDataManager implements IntermediaryDataManager try { deleteExpiredSupervisorTaskPartitionsIfNotRunning(); } - catch (InterruptedException e) { - LOG.error(e, "Error while cleaning up partitions for expired supervisors"); - } catch (Exception e) { LOG.warn(e, "Error while cleaning up partitions for expired supervisors"); } @@ -236,7 +234,7 @@ public class LocalIntermediaryDataManager implements IntermediaryDataManager * Note that the overlord sends a cleanup request when a supervisorTask is finished. The below check is to trigger * the self-cleanup for when the cleanup request is missing. */ - private void deleteExpiredSupervisorTaskPartitionsIfNotRunning() throws InterruptedException + private void deleteExpiredSupervisorTaskPartitionsIfNotRunning() { final Set expiredSupervisorTasks = new HashSet<>(); for (Entry entry : supervisorTaskCheckTimes.entrySet()) { @@ -252,7 +250,8 @@ public class LocalIntermediaryDataManager implements IntermediaryDataManager } if (!expiredSupervisorTasks.isEmpty()) { - final Map taskStatuses = indexingServiceClient.getTaskStatuses(expiredSupervisorTasks); + final Map taskStatuses = + FutureUtils.getUnchecked(overlordClient.taskStatuses(expiredSupervisorTasks), true); for (Entry entry : taskStatuses.entrySet()) { final String supervisorTaskId = entry.getKey(); final TaskStatus status = entry.getValue(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index 3f35db0f8f6..3e312990038 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; @@ -153,7 +153,7 @@ public class TaskToolboxTest new NoopChatHandlerProvider(), new DropwizardRowIngestionMetersFactory(), new TestAppenderatorsManager(), - new NoopIndexingServiceClient(), + new NoopOverlordClient(), null, null, null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java index f2b8f5b5025..abb4273cd41 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java @@ -27,21 +27,21 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.base.Stopwatch; import com.google.common.collect.ImmutableMap; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.NoopInputSource; import org.apache.druid.guice.DruidSecondaryModule; import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.indexing.common.task.NoopIndexTaskClientFactory; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexMergerV9Factory; @@ -65,8 +65,11 @@ import java.util.concurrent.TimeUnit; */ public class TestUtils { - public static final IndexingServiceClient INDEXING_SERVICE_CLIENT = new NoopIndexingServiceClient(); - public static final IndexTaskClientFactory TASK_CLIENT_FACTORY = new NoopIndexTaskClientFactory<>(); + public static final OverlordClient OVERLORD_SERVICE_CLIENT = new NoopOverlordClient(); + public static final ParallelIndexSupervisorTaskClientProvider TASK_CLIENT_PROVIDER = + (supervisorTaskId, httpTimeout, numRetries) -> { + throw new UnsupportedOperationException(); + }; public static final AppenderatorsManager APPENDERATORS_MANAGER = new TestAppenderatorsManager(); private static final Logger log = new Logger(TestUtils.class); @@ -101,11 +104,11 @@ public class TestUtils .addValue(AuthorizerMapper.class, null) .addValue(RowIngestionMetersFactory.class, rowIngestionMetersFactory) .addValue(PruneSpecsHolder.class, PruneSpecsHolder.DEFAULT) - .addValue(IndexingServiceClient.class, INDEXING_SERVICE_CLIENT) + .addValue(IndexingServiceClient.class, OVERLORD_SERVICE_CLIENT) .addValue(AuthorizerMapper.class, new AuthorizerMapper(ImmutableMap.of())) .addValue(AppenderatorsManager.class, APPENDERATORS_MANAGER) .addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller()) - .addValue(IndexTaskClientFactory.class, TASK_CLIENT_FACTORY) + .addValue(IndexTaskClientFactory.class, TASK_CLIENT_PROVIDER) ); jsonMapper.registerModule( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java index c3528c72c02..f93ae485257 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java @@ -20,9 +20,6 @@ package org.apache.druid.indexing.common.actions; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.discovery.DruidLeaderClient; -import org.apache.druid.indexing.common.RetryPolicyConfig; -import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; @@ -30,13 +27,19 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder; import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.apache.druid.rpc.HttpResponseException; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; import org.easymock.EasyMock; import org.jboss.netty.buffer.BigEndianHeapChannelBuffer; +import org.jboss.netty.handler.codec.http.DefaultHttpResponse; import org.jboss.netty.handler.codec.http.HttpMethod; 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 org.junit.Before; import org.junit.Rule; @@ -44,36 +47,32 @@ import org.junit.Test; import org.junit.rules.ExpectedException; import java.io.IOException; -import java.net.URL; import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; public class RemoteTaskActionClientTest { @Rule public ExpectedException expectedException = ExpectedException.none(); - private DruidLeaderClient druidLeaderClient; + private ServiceClient directOverlordClient; private final ObjectMapper objectMapper = new DefaultObjectMapper(); @Before public void setUp() { - druidLeaderClient = EasyMock.createMock(DruidLeaderClient.class); + directOverlordClient = EasyMock.createMock(ServiceClient.class); } @Test public void testSubmitSimple() throws Exception { - Request request = new Request(HttpMethod.POST, new URL("http://localhost:1234/xx")); - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/action")) - .andReturn(request); - - // return status code 200 and a list with size equals 1 - Map responseBody = new HashMap<>(); + // return OK response and a list with size equals 1 + final Map expectedResponse = new HashMap<>(); final List expectedLocks = Collections.singletonList(new TimeChunkLock( TaskLockType.SHARED, "groupId", @@ -82,68 +81,71 @@ public class RemoteTaskActionClientTest "version", 0 )); - responseBody.put("result", expectedLocks); - String strResult = objectMapper.writeValueAsString(responseBody); - final HttpResponse response = EasyMock.createNiceMock(HttpResponse.class); - EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0)); - EasyMock.replay(response); - StringFullResponseHolder responseHolder = new StringFullResponseHolder( - response, - StandardCharsets.UTF_8 - ).addChunk(strResult); + expectedResponse.put("result", expectedLocks); - // set up mocks - EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder); - EasyMock.replay(druidLeaderClient); + final DefaultHttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + final BytesFullResponseHolder responseHolder = new BytesFullResponseHolder(httpResponse); + responseHolder.addChunk(objectMapper.writeValueAsBytes(expectedResponse)); - Task task = NoopTask.create("id", 0); - RemoteTaskActionClient client = new RemoteTaskActionClient( - task, - druidLeaderClient, - new RetryPolicyFactory(new RetryPolicyConfig()), - objectMapper - ); - final List locks = client.submit(new LockListAction()); + final Task task = NoopTask.create("id", 0); + final LockListAction action = new LockListAction(); + + EasyMock.expect( + directOverlordClient.request( + EasyMock.eq( + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/action") + .jsonContent(objectMapper, new TaskActionHolder(task, action))), + EasyMock.anyObject(BytesFullResponseHandler.class) + ) + ) + .andReturn(responseHolder); + + EasyMock.replay(directOverlordClient); + + RemoteTaskActionClient client = new RemoteTaskActionClient(task, directOverlordClient, objectMapper); + final List locks = client.submit(action); Assert.assertEquals(expectedLocks, locks); - EasyMock.verify(druidLeaderClient); + EasyMock.verify(directOverlordClient); } @Test public void testSubmitWithIllegalStatusCode() throws Exception { - // return status code 400 and a list with size equals 1 - Request request = new Request(HttpMethod.POST, new URL("http://localhost:1234/xx")); - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/action")) - .andReturn(request); - - // return status code 200 and a list with size equals 1 + // return status code 400 final HttpResponse response = EasyMock.createNiceMock(HttpResponse.class); EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).anyTimes(); EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0)); EasyMock.replay(response); + StringFullResponseHolder responseHolder = new StringFullResponseHolder( response, StandardCharsets.UTF_8 ).addChunk("testSubmitWithIllegalStatusCode"); - // set up mocks - EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder); - EasyMock.replay(druidLeaderClient); + final Task task = NoopTask.create("id", 0); + final LockListAction action = new LockListAction(); + EasyMock.expect( + directOverlordClient.request( + EasyMock.eq( + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/action") + .jsonContent(objectMapper, new TaskActionHolder(task, action)) + ), + EasyMock.anyObject(BytesFullResponseHandler.class) + ) + ) + .andThrow(new ExecutionException(new HttpResponseException(responseHolder))); - Task task = NoopTask.create("id", 0); - RemoteTaskActionClient client = new RemoteTaskActionClient( - task, - druidLeaderClient, - new RetryPolicyFactory(objectMapper.readValue("{\"maxRetryCount\":0}", RetryPolicyConfig.class)), - objectMapper - ); + EasyMock.replay(directOverlordClient); + + RemoteTaskActionClient client = new RemoteTaskActionClient(task, directOverlordClient, objectMapper); expectedException.expect(IOException.class); expectedException.expectMessage( "Error with status[400 Bad Request] and message[testSubmitWithIllegalStatusCode]. " + "Check overlord logs for details." ); - client.submit(new LockListAction()); + client.submit(action); + + EasyMock.verify(directOverlordClient, response); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 01012ac350f..ecae743374f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -31,7 +31,7 @@ import org.apache.commons.io.FileUtils; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; @@ -1654,7 +1654,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand new NoopChatHandlerProvider(), testUtils.getRowIngestionMetersFactory(), new TestAppenderatorsManager(), - new NoopIndexingServiceClient(), + new NoopOverlordClient(), null, null, null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index 011b35eb634..c6b6d6fdf0a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -33,8 +33,7 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.client.indexing.ClientTaskQuery; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.SegmentsSplitHintSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.guice.GuiceAnnotationIntrospector; @@ -53,6 +52,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.BitmapSerde.DefaultBitmapSerdeFactory; import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; @@ -382,7 +382,7 @@ public class ClientCompactionTaskQuerySerdeTest binder.bind(CoordinatorClient.class).toInstance(COORDINATOR_CLIENT); binder.bind(SegmentCacheManagerFactory.class).toInstance(new SegmentCacheManagerFactory(objectMapper)); binder.bind(AppenderatorsManager.class).toInstance(APPENDERATORS_MANAGER); - binder.bind(IndexingServiceClient.class).toInstance(new NoopIndexingServiceClient()); + binder.bind(OverlordClient.class).toInstance(new NoopOverlordClient()); } ) ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index b4cf217d8f6..b0ddf61c560 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -28,8 +28,7 @@ import com.google.common.io.Files; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.ParseSpec; @@ -65,6 +64,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; @@ -164,7 +164,7 @@ public class CompactionTaskRunTest extends IngestionTestBase private static final String DATA_SOURCE = "test"; private static final RetryPolicyFactory RETRY_POLICY_FACTORY = new RetryPolicyFactory(new RetryPolicyConfig()); - private final IndexingServiceClient indexingServiceClient; + private final OverlordClient overlordClient; private final CoordinatorClient coordinatorClient; private final SegmentCacheManagerFactory segmentCacheManagerFactory; private final LockGranularity lockGranularity; @@ -176,7 +176,7 @@ public class CompactionTaskRunTest extends IngestionTestBase public CompactionTaskRunTest(LockGranularity lockGranularity) { testUtils = new TestUtils(); - indexingServiceClient = new NoopIndexingServiceClient(); + overlordClient = new NoopOverlordClient(); coordinatorClient = new CoordinatorClient(null, null) { @Override @@ -1713,7 +1713,7 @@ public class CompactionTaskRunTest extends IngestionTestBase .chatHandlerProvider(new NoopChatHandlerProvider()) .rowIngestionMetersFactory(testUtils.getRowIngestionMetersFactory()) .appenderatorsManager(new TestAppenderatorsManager()) - .indexingServiceClient(indexingServiceClient) + .overlordClient(overlordClient) .coordinatorClient(coordinatorClient) .build(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index b88cf10a3a2..e1e8a80f7e4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -38,8 +38,6 @@ import com.google.common.collect.Maps; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.common.guava.SettableSupplier; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionSchema; @@ -184,7 +182,6 @@ public class CompactionTaskTest private static final TestUtils TEST_UTILS = new TestUtils(); private static final Map SEGMENT_MAP = new HashMap<>(); private static final CoordinatorClient COORDINATOR_CLIENT = new TestCoordinatorClient(SEGMENT_MAP); - private static final IndexingServiceClient INDEXING_SERVICE_CLIENT = new NoopIndexingServiceClient(); private static final ObjectMapper OBJECT_MAPPER = setupInjectablesInObjectMapper(new DefaultObjectMapper()); private static final RetryPolicyFactory RETRY_POLICY_FACTORY = new RetryPolicyFactory(new RetryPolicyConfig()); private static final String CONFLICTING_SEGMENT_GRANULARITY_FORMAT = @@ -288,7 +285,6 @@ public class CompactionTaskTest binder.bind(SegmentCacheManagerFactory.class) .toInstance(new SegmentCacheManagerFactory(objectMapper)); binder.bind(AppenderatorsManager.class).toInstance(new TestAppenderatorsManager()); - binder.bind(IndexingServiceClient.class).toInstance(INDEXING_SERVICE_CLIENT); } ) ) @@ -2049,7 +2045,6 @@ public class CompactionTaskTest .chatHandlerProvider(new NoopChatHandlerProvider()) .rowIngestionMetersFactory(TEST_UTILS.getRowIngestionMetersFactory()) .appenderatorsManager(new TestAppenderatorsManager()) - .indexingServiceClient(INDEXING_SERVICE_CLIENT) .coordinatorClient(COORDINATOR_CLIENT) .segmentCacheManager(segmentCacheManager) .build(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 1de41f57733..c6d10f2b87e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; @@ -347,7 +346,6 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest .chatHandlerProvider(new NoopChatHandlerProvider()) .rowIngestionMetersFactory(testUtils.getRowIngestionMetersFactory()) .appenderatorsManager(new TestAppenderatorsManager()) - .indexingServiceClient(new NoopIndexingServiceClient()) .build(); if (task.isReady(box.getTaskActionClient())) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 741ed298b52..01232c3875a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -30,7 +30,7 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -1006,7 +1006,7 @@ public class RealtimeIndexTaskTest extends InitializedNullHandlingTest new NoopChatHandlerProvider(), testUtils.getRowIngestionMetersFactory(), new TestAppenderatorsManager(), - new NoopIndexingServiceClient(), + new NoopOverlordClient(), null, null, null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index 145098d3362..41025507b6f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.google.common.base.Preconditions; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LocalInputSource; @@ -187,7 +188,7 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn Map runTaskAndGetReports(Task task, TaskState expectedTaskStatus) { runTaskAndVerifyStatus(task, expectedTaskStatus); - return getIndexingServiceClient().getTaskReport(task.getId()); + return FutureUtils.getUnchecked(getIndexingServiceClient().taskReportAsMap(task.getId()), true); } protected ParallelIndexSupervisorTask createTask( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 0bf1021a891..40094e3445c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -33,7 +33,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.MaxSizeSplitHintSpec; @@ -51,14 +51,12 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.task.CompactionTask; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IngestionTestBase; import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.indexing.common.task.Task; @@ -222,7 +220,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase private File localDeepStorage; private SimpleThreadingTaskRunner taskRunner; private ObjectMapper objectMapper; - private LocalIndexingServiceClient indexingServiceClient; + private LocalOverlordClient indexingServiceClient; private IntermediaryDataManager intermediaryDataManager; private CoordinatorClient coordinatorClient; // An executor that executes API calls using a different thread from the caller thread as if they were remote calls. @@ -243,7 +241,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase localDeepStorage = temporaryFolder.newFolder("localStorage"); taskRunner = new SimpleThreadingTaskRunner(); objectMapper = getObjectMapper(); - indexingServiceClient = new LocalIndexingServiceClient(objectMapper, taskRunner); + indexingServiceClient = new LocalOverlordClient(objectMapper, taskRunner); intermediaryDataManager = new LocalIntermediaryDataManager( new WorkerConfig(), new TaskConfig( @@ -317,7 +315,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase ); } - protected LocalIndexingServiceClient getIndexingServiceClient() + protected LocalOverlordClient getIndexingServiceClient() { return indexingServiceClient; } @@ -538,32 +536,33 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase } } - public class LocalIndexingServiceClient extends NoopIndexingServiceClient + public class LocalOverlordClient extends NoopOverlordClient { private final ObjectMapper objectMapper; private final SimpleThreadingTaskRunner taskRunner; - public LocalIndexingServiceClient(ObjectMapper objectMapper, SimpleThreadingTaskRunner taskRunner) + public LocalOverlordClient(ObjectMapper objectMapper, SimpleThreadingTaskRunner taskRunner) { this.objectMapper = objectMapper; this.taskRunner = taskRunner; } @Override - public String runTask(String taskId, Object taskObject) + public ListenableFuture runTask(String taskId, Object taskObject) { final Task task = (Task) taskObject; - return taskRunner.run(injectIfNeeded(task)); + taskRunner.run(injectIfNeeded(task)); + return Futures.immediateFuture(null); } @Override - public Map getTaskReport(String taskId) + public ListenableFuture> taskReportAsMap(String taskId) { final Optional task = getTaskStorage().getTask(taskId); if (!task.isPresent()) { return null; } - return ((ParallelIndexSupervisorTask) task.get()).doGetLiveReports("full"); + return Futures.immediateFuture(((ParallelIndexSupervisorTask) task.get()).doGetLiveReports("full")); } public TaskContainer getTaskContainer(String taskId) @@ -598,13 +597,14 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase } @Override - public String cancelTask(String taskId) + public ListenableFuture cancelTask(String taskId) { - return taskRunner.cancel(taskId); + taskRunner.cancel(taskId); + return Futures.immediateFuture(null); } @Override - public TaskStatusResponse getTaskStatus(String taskId) + public ListenableFuture taskStatus(String taskId) { final Optional task = getTaskStorage().getTask(taskId); final String groupId = task.isPresent() ? task.get().getGroupId() : null; @@ -612,7 +612,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase final TaskStatus taskStatus = taskRunner.getStatus(taskId); if (taskStatus != null) { - return new TaskStatusResponse( + final TaskStatusResponse retVal = new TaskStatusResponse( taskId, new TaskStatusPlus( taskId, @@ -628,8 +628,10 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase null ) ); + + return Futures.immediateFuture(retVal); } else { - return new TaskStatusResponse(taskId, null); + return Futures.immediateFuture(new TaskStatusResponse(taskId, null)); } } @@ -734,9 +736,9 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase .chatHandlerProvider(new NoopChatHandlerProvider()) .rowIngestionMetersFactory(new TestUtils().getRowIngestionMetersFactory()) .appenderatorsManager(new TestAppenderatorsManager()) - .indexingServiceClient(indexingServiceClient) + .overlordClient(indexingServiceClient) .coordinatorClient(coordinatorClient) - .supervisorTaskClientFactory(new LocalParallelIndexTaskClientFactory(taskRunner, transientApiCallFailureRate)) + .supervisorTaskClientProvider(new LocalParallelIndexTaskClientProvider(taskRunner, transientApiCallFailureRate)) .shuffleClient(new LocalShuffleClient(intermediaryDataManager)) .build(); } @@ -909,50 +911,46 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase Assert.assertEquals(expectedInputs, actualInputs); } - static class LocalParallelIndexTaskClientFactory implements IndexTaskClientFactory + static class LocalParallelIndexTaskClientProvider implements ParallelIndexSupervisorTaskClientProvider { private final ConcurrentMap tasks; private final double transientApiCallFailureRate; - LocalParallelIndexTaskClientFactory(SimpleThreadingTaskRunner taskRunner, double transientApiCallFailureRate) + LocalParallelIndexTaskClientProvider(SimpleThreadingTaskRunner taskRunner, double transientApiCallFailureRate) { this.tasks = taskRunner.tasks; this.transientApiCallFailureRate = transientApiCallFailureRate; } + @Override - public ParallelIndexSupervisorTaskClient build( - TaskInfoProvider taskInfoProvider, - String callerId, - int numThreads, - Duration httpTimeout, - long numRetries - ) + public ParallelIndexSupervisorTaskClient build(String supervisorTaskId, Duration httpTimeout, long numRetries) { - return new LocalParallelIndexSupervisorTaskClient(callerId, tasks, transientApiCallFailureRate); + return new LocalParallelIndexSupervisorTaskClient(supervisorTaskId, tasks, transientApiCallFailureRate); } } - static class LocalParallelIndexSupervisorTaskClient extends ParallelIndexSupervisorTaskClient + static class LocalParallelIndexSupervisorTaskClient implements ParallelIndexSupervisorTaskClient { private static final int MAX_TRANSIENT_API_FAILURES = 3; + private final String supervisorTaskId; private final double transientFailureRate; private final ConcurrentMap tasks; LocalParallelIndexSupervisorTaskClient( - String callerId, + String supervisorTaskId, ConcurrentMap tasks, double transientFailureRate ) { - super(null, null, null, null, callerId, 0); + this.supervisorTaskId = supervisorTaskId; this.tasks = tasks; this.transientFailureRate = transientFailureRate; } @Override - public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException + public SegmentIdWithShardSpec allocateSegment(DateTime timestamp) throws IOException { final TaskContainer taskContainer = tasks.get(supervisorTaskId); final ParallelIndexSupervisorTask supervisorTask = findSupervisorTask(taskContainer); @@ -969,7 +967,6 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase @Override public SegmentIdWithShardSpec allocateSegment( - String supervisorTaskId, DateTime timestamp, String sequenceName, @Nullable String prevSegmentId @@ -1011,7 +1008,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase } @Override - public void report(String supervisorTaskId, SubTaskReport report) + public void report(SubTaskReport report) { final TaskContainer taskContainer = tasks.get(supervisorTaskId); final ParallelIndexSupervisorTask supervisorTask = findSupervisorTask(taskContainer); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index a187e9df8db..6e00b14c670 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -54,6 +54,7 @@ import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; +import org.joda.time.Duration; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -668,11 +669,9 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd } // build LocalParallelIndexTaskClient - final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientFactory().build( - null, - getId(), - 0, - null, + final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( + getSupervisorTaskId(), + Duration.ZERO, 0 ); final DynamicPartitionsSpec partitionsSpec = (DynamicPartitionsSpec) getIngestionSchema() @@ -709,7 +708,6 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd ); taskClient.report( - getSupervisorTaskId(), new PushedSegmentsReport(getId(), Collections.emptySet(), Collections.singleton(segment), ImmutableMap.of()) ); return TaskStatus.fromCode(getId(), state); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index 41e9c676d39..21291455656 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -22,6 +22,8 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Ordering; +import com.google.common.util.concurrent.Futures; +import org.apache.commons.codec.Charsets; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InlineInputSource; @@ -31,6 +33,9 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.apache.druid.rpc.HttpResponseException; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; @@ -41,13 +46,19 @@ import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.DimensionRangeBucketShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; import org.easymock.EasyMock; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; +import org.jboss.netty.buffer.ChannelBuffers; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Duration; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.runners.Enclosed; +import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -59,6 +70,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -383,6 +395,74 @@ public class ParallelIndexSupervisorTaskTest 1, task1); } + @Test + public void testGetTaskReportOk() throws Exception + { + final String taskId = "task"; + final Map report = ImmutableMap.of("foo", "bar"); + + final OverlordClient client = mock(OverlordClient.class); + expect(client.taskReportAsMap(taskId)).andReturn(Futures.immediateFuture(report)); + EasyMock.replay(client); + + Assert.assertEquals(report, ParallelIndexSupervisorTask.getTaskReport(client, taskId)); + EasyMock.verify(client); + } + + @Test + public void testGetTaskReport404() throws Exception + { + final String taskId = "task"; + + final OverlordClient client = mock(OverlordClient.class); + final HttpResponse response = mock(HttpResponse.class); + expect(response.getContent()).andReturn(ChannelBuffers.buffer(0)); + expect(response.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); + EasyMock.replay(response); + + expect(client.taskReportAsMap(taskId)).andReturn( + Futures.immediateFailedFuture( + new HttpResponseException(new StringFullResponseHolder(response, Charsets.UTF_8)) + ) + ); + EasyMock.replay(client); + + Assert.assertNull(ParallelIndexSupervisorTask.getTaskReport(client, taskId)); + EasyMock.verify(client, response); + } + + @Test + public void testGetTaskReport403() + { + final String taskId = "task"; + + final OverlordClient client = mock(OverlordClient.class); + final HttpResponse response = mock(HttpResponse.class); + expect(response.getContent()).andReturn(ChannelBuffers.buffer(0)); + expect(response.getStatus()).andReturn(HttpResponseStatus.FORBIDDEN).anyTimes(); + EasyMock.replay(response); + + expect(client.taskReportAsMap(taskId)).andReturn( + Futures.immediateFailedFuture( + new HttpResponseException(new StringFullResponseHolder(response, Charsets.UTF_8)) + ) + ); + EasyMock.replay(client); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> ParallelIndexSupervisorTask.getTaskReport(client, taskId) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(HttpResponseException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Server error [403 Forbidden]")) + ); + + EasyMock.verify(client, response); + } + private PartitionStat createRangePartitionStat(Interval interval, int bucketId) { return new DeepStoragePartitionStat( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index f080121f71b..150d0b2c180 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -33,7 +32,6 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -70,9 +68,7 @@ class ParallelIndexTestingFactory static final String SUBTASK_SPEC_ID = "subtask-spec-id"; static final int NUM_ATTEMPTS = 1; static final Map CONTEXT = Collections.emptyMap(); - static final IndexingServiceClient INDEXING_SERVICE_CLIENT = TestUtils.INDEXING_SERVICE_CLIENT; - static final IndexTaskClientFactory TASK_CLIENT_FACTORY = - TestUtils.TASK_CLIENT_FACTORY; + static final ParallelIndexSupervisorTaskClientProvider TASK_CLIENT_PROVIDER = TestUtils.TASK_CLIENT_PROVIDER; static final AppenderatorsManager APPENDERATORS_MANAGER = TestUtils.APPENDERATORS_MANAGER; static final ShuffleClient SHUFFLE_CLIENT = new ShuffleClient() { @@ -254,9 +250,9 @@ class ParallelIndexTestingFactory } } - static IndexTaskClientFactory createTaskClientFactory() + static ParallelIndexSupervisorTaskClientProvider createTaskClientFactory() { - return (taskInfoProvider, callerId, numThreads, httpTimeout, numRetries) -> createTaskClient(); + return (supervisorTaskId, httpTimeout, numRetries) -> createTaskClient(); } private static ParallelIndexSupervisorTaskClient createTaskClient() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java index 90b7de605b3..1e529cbfe6b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java @@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.memory.Memory; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -36,10 +35,8 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; -import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -52,7 +49,6 @@ import org.apache.logging.log4j.core.LogEvent; import org.easymock.Capture; import org.easymock.EasyMock; import org.hamcrest.Matchers; -import org.joda.time.Duration; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -147,27 +143,13 @@ public class PartialDimensionCardinalityTaskTest { reportCapture = Capture.newInstance(); ParallelIndexSupervisorTaskClient taskClient = EasyMock.mock(ParallelIndexSupervisorTaskClient.class); - taskClient.report(EasyMock.eq(ParallelIndexTestingFactory.SUPERVISOR_TASK_ID), EasyMock.capture(reportCapture)); + taskClient.report(EasyMock.capture(reportCapture)); EasyMock.replay(taskClient); taskToolbox = EasyMock.mock(TaskToolbox.class); EasyMock.expect(taskToolbox.getIndexingTmpDir()).andStubReturn(temporaryFolder.getRoot()); - EasyMock.expect(taskToolbox.getSupervisorTaskClientFactory()).andReturn( - new IndexTaskClientFactory() - { - @Override - public ParallelIndexSupervisorTaskClient build( - TaskInfoProvider taskInfoProvider, - String callerId, - int numThreads, - Duration httpTimeout, - long numRetries - ) - { - return taskClient; - } - } - ); - EasyMock.expect(taskToolbox.getIndexingServiceClient()).andReturn(new NoopIndexingServiceClient()); + EasyMock.expect(taskToolbox.getSupervisorTaskClientProvider()) + .andReturn((supervisorTaskId, httpTimeout, numRetries) -> taskClient); + EasyMock.expect(taskToolbox.getOverlordClient()).andReturn(null); EasyMock.expect(taskToolbox.getRowIngestionMetersFactory()).andReturn(new DropwizardRowIngestionMetersFactory()); EasyMock.replay(taskToolbox); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTaskTest.java index f5985ae7083..49fb4041380 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTaskTest.java @@ -22,7 +22,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.Iterables; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.StringTuple; @@ -33,10 +32,8 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; -import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistribution; import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch; import org.apache.druid.java.util.common.StringUtils; @@ -49,7 +46,6 @@ import org.apache.logging.log4j.core.LogEvent; import org.easymock.Capture; import org.easymock.EasyMock; import org.hamcrest.Matchers; -import org.joda.time.Duration; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -149,27 +145,13 @@ public class PartialDimensionDistributionTaskTest { reportCapture = Capture.newInstance(); ParallelIndexSupervisorTaskClient taskClient = EasyMock.mock(ParallelIndexSupervisorTaskClient.class); - taskClient.report(EasyMock.eq(ParallelIndexTestingFactory.SUPERVISOR_TASK_ID), EasyMock.capture(reportCapture)); + taskClient.report(EasyMock.capture(reportCapture)); EasyMock.replay(taskClient); taskToolbox = EasyMock.mock(TaskToolbox.class); EasyMock.expect(taskToolbox.getIndexingTmpDir()).andStubReturn(temporaryFolder.getRoot()); - EasyMock.expect(taskToolbox.getSupervisorTaskClientFactory()).andReturn( - new IndexTaskClientFactory() - { - @Override - public ParallelIndexSupervisorTaskClient build( - TaskInfoProvider taskInfoProvider, - String callerId, - int numThreads, - Duration httpTimeout, - long numRetries - ) - { - return taskClient; - } - } - ); - EasyMock.expect(taskToolbox.getIndexingServiceClient()).andReturn(new NoopIndexingServiceClient()); + EasyMock.expect(taskToolbox.getSupervisorTaskClientProvider()) + .andReturn((supervisorTaskId, httpTimeout, numRetries) -> taskClient); + EasyMock.expect(taskToolbox.getOverlordClient()).andReturn(null); EasyMock.expect(taskToolbox.getRowIngestionMetersFactory()).andReturn(new DropwizardRowIngestionMetersFactory()); EasyMock.replay(taskToolbox); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java index 085f7b9c4c2..caff8cc81d7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java @@ -19,8 +19,9 @@ package org.apache.druid.indexing.common.task.batch.parallel; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.data.input.InputSplit; import org.apache.druid.indexer.RunnerTaskState; @@ -57,7 +58,7 @@ public class TaskMonitorTest private final ExecutorService taskRunner = Execs.multiThreaded(5, "task-monitor-test-%d"); private final ConcurrentMap tasks = new ConcurrentHashMap<>(); private final TaskMonitor monitor = new TaskMonitor<>( - new TestIndexingServiceClient(), + new TestOverlordClient(), 3, SPLIT_NUM ); @@ -247,10 +248,10 @@ public class TaskMonitorTest } } - private class TestIndexingServiceClient extends NoopIndexingServiceClient + private class TestOverlordClient extends NoopOverlordClient { @Override - public String runTask(String taskId, Object taskObject) + public ListenableFuture runTask(String taskId, Object taskObject) { final TestTask task = (TestTask) taskObject; tasks.put(task.getId(), TaskState.RUNNING); @@ -258,13 +259,13 @@ public class TaskMonitorTest throw new RuntimeException(new ISE("Could not resolve type id 'test_task_id'")); } taskRunner.submit(() -> tasks.put(task.getId(), task.run(null).getStatusCode())); - return task.getId(); + return Futures.immediateFuture(null); } @Override - public TaskStatusResponse getTaskStatus(String taskId) + public ListenableFuture taskStatus(String taskId) { - return new TaskStatusResponse( + final TaskStatusResponse retVal = new TaskStatusResponse( taskId, new TaskStatusPlus( taskId, @@ -280,6 +281,8 @@ public class TaskMonitorTest null ) ); + + return Futures.immediateFuture(retVal); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index fd8510be752..0278dca8bc1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -20,7 +20,7 @@ package org.apache.druid.indexing.overlord; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; @@ -135,7 +135,7 @@ public class SingleTaskBackgroundRunnerTest new NoopChatHandlerProvider(), utils.getRowIngestionMetersFactory(), new TestAppenderatorsManager(), - new NoopIndexingServiceClient(), + new NoopOverlordClient(), null, null, null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 980300e9f09..bb5a6d2ae45 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -33,7 +33,7 @@ import com.google.common.collect.Ordering; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.AbstractInputSource; import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; @@ -696,7 +696,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest new NoopChatHandlerProvider(), TEST_UTILS.getRowIngestionMetersFactory(), appenderatorsManager, - new NoopIndexingServiceClient(), + new NoopOverlordClient(), null, null, null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index 2f635b76829..80c734e0bfe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -22,7 +22,7 @@ package org.apache.druid.indexing.worker; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -139,7 +139,7 @@ public class WorkerTaskManagerTest new NoopChatHandlerProvider(), testUtils.getRowIngestionMetersFactory(), new TestAppenderatorsManager(), - new NoopIndexingServiceClient(), + new NoopOverlordClient(), null, null, null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 63a072d5399..b6261db8983 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -26,7 +26,7 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.TestingCluster; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.curator.PotentiallyGzippedCompressionProvider; import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.indexer.TaskState; @@ -209,7 +209,7 @@ public class WorkerTaskMonitorTest new NoopChatHandlerProvider(), testUtils.getRowIngestionMetersFactory(), new TestAppenderatorsManager(), - new NoopIndexingServiceClient(), + new NoopOverlordClient(), null, null, null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java index 6120a884420..9b3a258df6d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java @@ -21,14 +21,16 @@ package org.apache.druid.indexing.worker.shuffle; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.commons.io.FileUtils; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.client.indexing.TaskStatus; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.BucketNumberedShardSpec; @@ -57,7 +59,7 @@ public class LocalIntermediaryDataManagerAutoCleanupTest public TemporaryFolder tempDir = new TemporaryFolder(); private TaskConfig taskConfig; - private IndexingServiceClient indexingServiceClient; + private OverlordClient overlordClient; @Before public void setup() throws IOException @@ -77,17 +79,17 @@ public class LocalIntermediaryDataManagerAutoCleanupTest TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), null ); - this.indexingServiceClient = new NoopIndexingServiceClient() + this.overlordClient = new NoopOverlordClient() { @Override - public Map getTaskStatuses(Set taskIds) + public ListenableFuture> taskStatuses(Set taskIds) { final Map result = new HashMap<>(); for (String taskId : taskIds) { TaskState state = taskId.startsWith("running_") ? TaskState.RUNNING : TaskState.SUCCESS; result.put(taskId, new TaskStatus(taskId, state, 10)); } - return result; + return Futures.immediateFuture(result); } }; } @@ -133,7 +135,7 @@ public class LocalIntermediaryDataManagerAutoCleanupTest // Setup data manager with expiry timeout 1s WorkerConfig workerConfig = new TestWorkerConfig(1, 1, timeoutPeriod); LocalIntermediaryDataManager intermediaryDataManager = - new LocalIntermediaryDataManager(workerConfig, taskConfig, indexingServiceClient); + new LocalIntermediaryDataManager(workerConfig, taskConfig, overlordClient); intermediaryDataManager.addSegment(supervisorTaskId, subTaskId, segment, segmentFile); intermediaryDataManager diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java index 526328445be..65eeee9af51 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java @@ -23,12 +23,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.io.ByteSource; import com.google.common.primitives.Ints; import org.apache.commons.io.FileUtils; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.BucketNumberedShardSpec; @@ -83,8 +83,8 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), null ); - final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient(); - intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, indexingServiceClient); + final OverlordClient overlordClient = new NoopOverlordClient(); + intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, overlordClient); intermediaryDataManager.start(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java index b4cd119c315..c318e85c137 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java @@ -28,8 +28,7 @@ import com.google.common.io.Files; import com.google.common.primitives.Ints; import com.google.inject.Injector; import org.apache.commons.io.FileUtils; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.guice.GuiceAnnotationIntrospector; import org.apache.druid.guice.GuiceInjectableValues; import org.apache.druid.guice.GuiceInjectors; @@ -37,6 +36,7 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.loading.LoadSpec; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalDataSegmentPusher; @@ -113,9 +113,9 @@ public class ShuffleDataSegmentPusherTest TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), null ); - final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient(); + final OverlordClient overlordClient = new NoopOverlordClient(); if (LOCAL.equals(intermediateDataStore)) { - intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, indexingServiceClient); + intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, overlordClient); } else if (DEEPSTORE.equals(intermediateDataStore)) { localDeepStore = temporaryFolder.newFolder("localStorage"); intermediaryDataManager = new DeepStorageIntermediaryDataManager( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java index bb6f6fdcfc9..c795c473dad 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java @@ -21,15 +21,17 @@ package org.apache.druid.indexing.worker.shuffle; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.commons.io.FileUtils; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.client.indexing.TaskStatus; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.indexing.worker.shuffle.ShuffleMetrics.PerDatasourceShuffleMetrics; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.BucketNumberedShardSpec; @@ -103,19 +105,19 @@ public class ShuffleResourceTest TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), null ); - final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient() + final OverlordClient overlordClient = new NoopOverlordClient() { @Override - public Map getTaskStatuses(Set taskIds) + public ListenableFuture> taskStatuses(Set taskIds) { final Map result = new HashMap<>(); for (String taskId : taskIds) { result.put(taskId, new TaskStatus(taskId, TaskState.SUCCESS, 10)); } - return result; + return Futures.immediateFuture(result); } }; - intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, indexingServiceClient); + intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, overlordClient); shuffleMetrics = new ShuffleMetrics(); shuffleResource = new ShuffleResource(intermediaryDataManager, Optional.of(shuffleMetrics)); } diff --git a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java index b811f6a5bd1..58dd8ae2421 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java +++ b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java @@ -321,27 +321,6 @@ public class HttpIndexingServiceClient implements IndexingServiceClient } } - @Override - public Map getTaskStatuses(Set taskIds) throws InterruptedException - { - try { - final StringFullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/taskStatus") - .setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskIds)) - ); - - return jsonMapper.readValue( - responseHolder.getContent(), - new TypeReference>() - { - } - ); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - @Override @Nullable public TaskStatusPlus getLastCompleteTask() diff --git a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java index 000f13254ac..2658d57d237 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java @@ -21,6 +21,7 @@ package org.apache.druid.client.indexing; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -28,8 +29,12 @@ import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.List; import java.util.Map; -import java.util.Set; +/** + * High-level IndexingServiceClient client. + * + * New use cases should prefer {@link OverlordClient}. + */ public interface IndexingServiceClient { void killUnusedSegments(String idPrefix, String dataSource, Interval interval); @@ -71,8 +76,6 @@ public interface IndexingServiceClient TaskStatusResponse getTaskStatus(String taskId); - Map getTaskStatuses(Set taskIds) throws InterruptedException; - @Nullable TaskStatusPlus getLastCompleteTask(); @@ -89,6 +92,7 @@ public interface IndexingServiceClient * Intervals that are locked by Tasks higher than this * priority are returned. Tasks for datasources that * are not present in this Map are not returned. + * * @return Map from Datasource to List of Intervals locked by Tasks that have * priority strictly greater than the {@code minTaskPriority} for that datasource. */ diff --git a/server/src/main/java/org/apache/druid/initialization/Initialization.java b/server/src/main/java/org/apache/druid/initialization/Initialization.java index e9672795fad..8777aa236c6 100644 --- a/server/src/main/java/org/apache/druid/initialization/Initialization.java +++ b/server/src/main/java/org/apache/druid/initialization/Initialization.java @@ -62,6 +62,7 @@ import org.apache.druid.guice.security.EscalatorModule; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule; +import org.apache.druid.rpc.guice.ServiceClientModule; import org.apache.druid.segment.writeout.SegmentWriteOutMediumModule; import org.apache.druid.server.emitter.EmitterModule; import org.apache.druid.server.initialization.AuthenticatorMapperModule; @@ -429,7 +430,8 @@ public class Initialization new AuthorizerModule(), new AuthorizerMapperModule(), new StartupLoggingModule(), - new ExternalStorageAccessSecurityModule() + new ExternalStorageAccessSecurityModule(), + new ServiceClientModule() ); ModuleList actualModules = new ModuleList(baseInjector, nodeRoles); diff --git a/server/src/main/java/org/apache/druid/rpc/DiscoveryServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/DiscoveryServiceLocator.java new file mode 100644 index 00000000000..0c520f1927b --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/DiscoveryServiceLocator.java @@ -0,0 +1,161 @@ +/* + * 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.collect.ImmutableSet; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; + +/** + * A {@link ServiceLocator} that uses {@link DruidNodeDiscovery}. + */ +public class DiscoveryServiceLocator implements ServiceLocator +{ + private final DruidNodeDiscoveryProvider discoveryProvider; + private final NodeRole nodeRole; + private final DruidNodeDiscovery.Listener listener; + + @GuardedBy("this") + private boolean started = false; + + @GuardedBy("this") + private boolean initialized = false; + + @GuardedBy("this") + private boolean closed = false; + + @GuardedBy("this") + private final Set locations = new HashSet<>(); + + @GuardedBy("this") + private SettableFuture pendingFuture = null; + + @GuardedBy("this") + private DruidNodeDiscovery discovery = null; + + public DiscoveryServiceLocator(final DruidNodeDiscoveryProvider discoveryProvider, final NodeRole nodeRole) + { + this.discoveryProvider = discoveryProvider; + this.nodeRole = nodeRole; + this.listener = new Listener(); + } + + @Override + public ListenableFuture locate() + { + synchronized (this) { + if (closed) { + return Futures.immediateFuture(ServiceLocations.closed()); + } else if (initialized) { + return Futures.immediateFuture(ServiceLocations.forLocations(ImmutableSet.copyOf(locations))); + } else { + if (pendingFuture == null) { + pendingFuture = SettableFuture.create(); + } + + return Futures.nonCancellationPropagating(pendingFuture); + } + } + } + + @LifecycleStart + public void start() + { + synchronized (this) { + if (started || closed) { + throw new ISE("Cannot start once already started or closed"); + } else { + started = true; + this.discovery = discoveryProvider.getForNodeRole(nodeRole); + discovery.registerListener(listener); + } + } + } + + @Override + @LifecycleStop + public void close() + { + synchronized (this) { + // Idempotent: can call close() multiple times so long as start() has already been called. + if (started && !closed) { + if (discovery != null) { + discovery.removeListener(listener); + } + + if (pendingFuture != null) { + pendingFuture.set(ServiceLocations.closed()); + pendingFuture = null; + } + + closed = true; + } + } + } + + private class Listener implements DruidNodeDiscovery.Listener + { + @Override + public void nodesAdded(final Collection nodes) + { + synchronized (DiscoveryServiceLocator.this) { + for (final DiscoveryDruidNode node : nodes) { + locations.add(ServiceLocation.fromDruidNode(node.getDruidNode())); + } + } + } + + @Override + public void nodesRemoved(final Collection nodes) + { + synchronized (DiscoveryServiceLocator.this) { + for (final DiscoveryDruidNode node : nodes) { + locations.remove(ServiceLocation.fromDruidNode(node.getDruidNode())); + } + } + } + + @Override + public void nodeViewInitialized() + { + synchronized (DiscoveryServiceLocator.this) { + initialized = true; + + if (pendingFuture != null) { + pendingFuture.set(ServiceLocations.forLocations(ImmutableSet.copyOf(locations))); + pendingFuture = null; + } + } + } + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/HttpResponseException.java b/server/src/main/java/org/apache/druid/rpc/HttpResponseException.java new file mode 100644 index 00000000000..5964bd4ef8e --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/HttpResponseException.java @@ -0,0 +1,61 @@ +/* + * 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 org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; + +import javax.annotation.Nullable; + +/** + * Returned by {@link ServiceClient#asyncRequest} when a request has failed due to an HTTP response. + */ +public class HttpResponseException extends RpcException +{ + private final StringFullResponseHolder responseHolder; + + public HttpResponseException(final StringFullResponseHolder responseHolder) + { + super( + "Server error [%s]; %s", + responseHolder.getStatus(), + choppedBodyErrorMessage(responseHolder.getContent()) + ); + + this.responseHolder = responseHolder; + } + + public StringFullResponseHolder getResponse() + { + return responseHolder; + } + + static String choppedBodyErrorMessage(@Nullable final String responseContent) + { + if (responseContent == null || responseContent.isEmpty()) { + return "no body"; + } else if (responseContent.length() > 1000) { + final String choppedMessage = StringUtils.chop(responseContent, 1000); + return "first 1KB of body: " + choppedMessage; + } else { + return "body: " + responseContent; + } + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/IgnoreHttpResponseHandler.java b/server/src/main/java/org/apache/druid/rpc/IgnoreHttpResponseHandler.java new file mode 100644 index 00000000000..b2bfee7079a --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/IgnoreHttpResponseHandler.java @@ -0,0 +1,63 @@ +/* + * 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 org.apache.druid.java.util.http.client.response.ClientResponse; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.jboss.netty.handler.codec.http.HttpChunk; +import org.jboss.netty.handler.codec.http.HttpResponse; + +/** + * An HTTP response handler that discards the response and returns nothing. It returns a finished response only + * when the entire HTTP response is done. + */ +public class IgnoreHttpResponseHandler implements HttpResponseHandler +{ + public static final IgnoreHttpResponseHandler INSTANCE = new IgnoreHttpResponseHandler(); + + private IgnoreHttpResponseHandler() + { + // Singleton. + } + + @Override + public ClientResponse handleResponse(HttpResponse response, TrafficCop trafficCop) + { + return ClientResponse.unfinished(null); + } + + @Override + public ClientResponse handleChunk(ClientResponse clientResponse, HttpChunk chunk, long chunkNum) + { + return ClientResponse.unfinished(null); + } + + @Override + public ClientResponse done(ClientResponse clientResponse) + { + return ClientResponse.finished(null); + } + + @Override + public void exceptionCaught(ClientResponse clientResponse, Throwable e) + { + // Safe to ignore, since the ClientResponses returned in handleChunk were unfinished. + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/RequestBuilder.java b/server/src/main/java/org/apache/druid/rpc/RequestBuilder.java new file mode 100644 index 00000000000..bc75d848f5a --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/RequestBuilder.java @@ -0,0 +1,186 @@ +/* + * 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.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.http.client.Request; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.joda.time.Duration; + +import javax.ws.rs.core.MediaType; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Arrays; +import java.util.Map; +import java.util.Objects; + +/** + * Used by {@link ServiceClient} to generate {@link Request} objects for an + * {@link org.apache.druid.java.util.http.client.HttpClient}. + */ +public class RequestBuilder +{ + @VisibleForTesting + static final Duration DEFAULT_TIMEOUT = Duration.standardMinutes(2); + + private final HttpMethod method; + private final String encodedPathAndQueryString; + private final Multimap headers = HashMultimap.create(); + private String contentType = null; + private byte[] content = null; + private Duration timeout = DEFAULT_TIMEOUT; + + public RequestBuilder(final HttpMethod method, final String encodedPathAndQueryString) + { + this.method = Preconditions.checkNotNull(method, "method"); + this.encodedPathAndQueryString = Preconditions.checkNotNull(encodedPathAndQueryString, "encodedPathAndQueryString"); + + if (!encodedPathAndQueryString.startsWith("/")) { + throw new IAE("Path must start with '/'"); + } + } + + public RequestBuilder header(final String header, final String value) + { + headers.put(header, value); + return this; + } + + public RequestBuilder content(final String contentType, final byte[] content) + { + this.contentType = Preconditions.checkNotNull(contentType, "contentType"); + this.content = Preconditions.checkNotNull(content, "content"); + return this; + } + + public RequestBuilder jsonContent(final ObjectMapper jsonMapper, final Object content) + { + try { + this.contentType = MediaType.APPLICATION_JSON; + this.content = jsonMapper.writeValueAsBytes(Preconditions.checkNotNull(content, "content")); + return this; + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + public RequestBuilder smileContent(final ObjectMapper smileMapper, final Object content) + { + try { + this.contentType = SmileMediaTypes.APPLICATION_JACKSON_SMILE; + this.content = smileMapper.writeValueAsBytes(Preconditions.checkNotNull(content, "content")); + return this; + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + public RequestBuilder timeout(final Duration timeout) + { + this.timeout = Preconditions.checkNotNull(timeout, "timeout"); + return this; + } + + /** + * Accessor for request timeout. Provided because the timeout is not part of the request generated + * by {@link #build(ServiceLocation)}. + * + * If there is no timeout, returns an empty Duration. + */ + public Duration getTimeout() + { + return timeout; + } + + public Request build(ServiceLocation serviceLocation) + { + // It's expected that our encodedPathAndQueryString starts with '/' and the service base path doesn't end with one. + final String path = serviceLocation.getBasePath() + encodedPathAndQueryString; + final Request request = new Request(method, makeURL(serviceLocation, path)); + + for (final Map.Entry entry : headers.entries()) { + request.addHeader(entry.getKey(), entry.getValue()); + } + + if (contentType != null) { + request.setContent(contentType, content); + } + + return request; + } + + private URL makeURL(final ServiceLocation serviceLocation, final String encodedPathAndQueryString) + { + final String scheme; + final int portToUse; + + if (serviceLocation.getTlsPort() > 0) { + // Prefer HTTPS if available. + scheme = "https"; + portToUse = serviceLocation.getTlsPort(); + } else { + scheme = "http"; + portToUse = serviceLocation.getPlaintextPort(); + } + + // Use URL constructor, not URI, since the path is already encoded. + try { + return new URL(scheme, serviceLocation.getHost(), portToUse, encodedPathAndQueryString); + } + catch (MalformedURLException e) { + throw new IllegalArgumentException(e); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RequestBuilder that = (RequestBuilder) o; + return Objects.equals(method, that.method) + && Objects.equals(encodedPathAndQueryString, that.encodedPathAndQueryString) + && Objects.equals(headers, that.headers) + && Objects.equals(contentType, that.contentType) + && Arrays.equals(content, that.content) + && Objects.equals(timeout, that.timeout); + } + + @Override + public int hashCode() + { + int result = Objects.hash(method, encodedPathAndQueryString, headers, contentType, timeout); + result = 31 * result + Arrays.hashCode(content); + return result; + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/RpcException.java b/server/src/main/java/org/apache/druid/rpc/RpcException.java new file mode 100644 index 00000000000..47e95fe113f --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/RpcException.java @@ -0,0 +1,40 @@ +/* + * 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 org.apache.druid.java.util.common.StringUtils; + +import java.io.IOException; + +/** + * Returned by {@link ServiceClient#asyncRequest} when a request has failed. + */ +public class RpcException extends IOException +{ + public RpcException(String formatText, Object... arguments) + { + super(StringUtils.nonStrictFormat(formatText, arguments)); + } + + public RpcException(Throwable cause, String formatText, Object... arguments) + { + super(StringUtils.nonStrictFormat(formatText, arguments), cause); + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClient.java b/server/src/main/java/org/apache/druid/rpc/ServiceClient.java new file mode 100644 index 00000000000..458016c867b --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClient.java @@ -0,0 +1,97 @@ +/* + * 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.ListenableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.rpc.indexing.OverlordClient; + +import java.util.concurrent.ExecutionException; + +/** + * Mid-level client that provides an API similar to low-level {@link HttpClient}, but accepts {@link RequestBuilder} + * instead of {@link org.apache.druid.java.util.http.client.Request}, and internally handles service location + * and retries. + * + * In most cases, this client is further wrapped in a high-level client like + * {@link OverlordClient}. + */ +public interface ServiceClient +{ + long MAX_REDIRECTS = 3; + + /** + * Perform a request asynchronously. + * + * Unlike {@link HttpClient#go}, the provided "handler" is only used for 2xx responses. + * + * Response codes 1xx, 4xx, and 5xx are retried with backoff according to the client's {@link ServiceRetryPolicy}. + * If attempts are exhausted, the future will fail with {@link RpcException} containing the most recently + * encountered error. + * + * Redirects from 3xx responses are followed up to a chain length of {@link #MAX_REDIRECTS} and do not consume + * 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 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 + * provided {@link ServiceRetryPolicy}. On the other hand, if the handler has returned a finished object, the + * service client will not retry. Behavior in this case is up to the caller, who will have already received the + * finished object as the future's resolved value. + * + * Resolves to {@link HttpResponseException} if the final attempt failed due to a non-OK HTTP server response. + * + * Resolves to {@link ServiceNotAvailableException} if the final attempt failed because the service was not + * available (i.e. if the locator returned an empty set of locations). + * + * Resolves to {@link ServiceClosedException} if the final attempt failed because the service was closed. This is + * different from not available: generally, "not available" is a temporary condition whereas "closed" is a + * permanent condition. + */ + ListenableFuture asyncRequest( + RequestBuilder requestBuilder, + HttpResponseHandler handler + ); + + /** + * Perform a request synchronously. + * + * Same behavior as {@link #asyncRequest}, except the result is returned synchronously. Any exceptions from the + * underlying service call are wrapped in an ExecutionException. + */ + default FinalType request( + RequestBuilder requestBuilder, + HttpResponseHandler handler + ) throws InterruptedException, ExecutionException + { + // Cancel the future if we are interrupted. Nobody else is waiting for it. + return FutureUtils.get(asyncRequest(requestBuilder, handler), true); + } + + /** + * Returns a copy of this client with a different {@link ServiceRetryPolicy}. + */ + ServiceClient withRetryPolicy(ServiceRetryPolicy retryPolicy); +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClientFactory.java b/server/src/main/java/org/apache/druid/rpc/ServiceClientFactory.java new file mode 100644 index 00000000000..0df6041b7ba --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClientFactory.java @@ -0,0 +1,36 @@ +/* + * 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; + +/** + * Factory for creating {@link ServiceClient}. + */ +public interface ServiceClientFactory +{ + /** + * Creates a client for a particular service. + * + * @param serviceName name of the service, which is used in log messages and exceptions. + * @param serviceLocator service locator. This is not owned by the returned client, and should be closed + * separately when you are done with it. + * @param retryPolicy retry policy + */ + ServiceClient makeClient(String serviceName, ServiceLocator serviceLocator, ServiceRetryPolicy retryPolicy); +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClientFactoryImpl.java b/server/src/main/java/org/apache/druid/rpc/ServiceClientFactoryImpl.java new file mode 100644 index 00000000000..5c997f21194 --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClientFactoryImpl.java @@ -0,0 +1,52 @@ +/* + * 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 org.apache.druid.java.util.http.client.HttpClient; + +import java.util.concurrent.ScheduledExecutorService; + +/** + * Production implementation of {@link ServiceClientFactory}. + */ +public class ServiceClientFactoryImpl implements ServiceClientFactory +{ + private final HttpClient httpClient; + private final ScheduledExecutorService connectExec; + + public ServiceClientFactoryImpl( + final HttpClient httpClient, + final ScheduledExecutorService connectExec + ) + { + this.httpClient = httpClient; + this.connectExec = connectExec; + } + + @Override + public ServiceClient makeClient( + final String serviceName, + final ServiceLocator serviceLocator, + final ServiceRetryPolicy retryPolicy + ) + { + return new ServiceClientImpl(serviceName, httpClient, serviceLocator, retryPolicy, connectExec); + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java b/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java new file mode 100644 index 00000000000..9a2ce234c5e --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java @@ -0,0 +1,424 @@ +/* + * 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.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +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.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.java.util.http.client.response.ObjectOrErrorResponseHandler; +import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import javax.annotation.Nullable; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +/** + * Production implementation of {@link ServiceClient}. + */ +public class ServiceClientImpl implements ServiceClient +{ + private static final Logger log = new Logger(ServiceClientImpl.class); + + private final String serviceName; + private final HttpClient httpClient; + private final ServiceLocator serviceLocator; + private final ServiceRetryPolicy retryPolicy; + private final ScheduledExecutorService connectExec; + + // Populated when we receive a redirect. The location here has no base path; it only identifies a preferred server. + private final AtomicReference preferredLocationNoPath = new AtomicReference<>(); + + public ServiceClientImpl( + final String serviceName, + final HttpClient httpClient, + final ServiceLocator serviceLocator, + final ServiceRetryPolicy retryPolicy, + final ScheduledExecutorService connectExec + ) + { + this.serviceName = Preconditions.checkNotNull(serviceName, "serviceName"); + this.httpClient = Preconditions.checkNotNull(httpClient, "httpClient"); + this.serviceLocator = Preconditions.checkNotNull(serviceLocator, "serviceLocator"); + this.retryPolicy = Preconditions.checkNotNull(retryPolicy, "retryPolicy"); + this.connectExec = Preconditions.checkNotNull(connectExec, "connectExec"); + + if (retryPolicy.maxAttempts() == 0) { + throw new IAE("Invalid maxAttempts[%d] in retry policy", retryPolicy.maxAttempts()); + } + } + + @Override + public ListenableFuture asyncRequest( + final RequestBuilder requestBuilder, + final HttpResponseHandler handler + ) + { + final SettableFuture retVal = SettableFuture.create(); + tryRequest(requestBuilder, handler, retVal, 0, 0); + return retVal; + } + + @Override + public ServiceClientImpl withRetryPolicy(ServiceRetryPolicy newRetryPolicy) + { + return new ServiceClientImpl(serviceName, httpClient, serviceLocator, newRetryPolicy, connectExec); + } + + private void tryRequest( + final RequestBuilder requestBuilder, + final HttpResponseHandler handler, + final SettableFuture retVal, + final long attemptNumber, + final int redirectCount + ) + { + whenServiceReady( + serviceLocation -> { + if (retVal.isCancelled()) { + // Return early if the caller canceled the return future. + return; + } + + 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); + + if (shouldTry(nextAttemptNumber)) { + log.info( + "Service [%s] not available on attempt #%d; retrying in %,d ms.", + serviceName, + nextAttemptNumber, + backoffMs + ); + + connectExec.schedule( + () -> tryRequest(requestBuilder, handler, retVal, attemptNumber + 1, redirectCount), + backoffMs, + TimeUnit.MILLISECONDS + ); + } else { + retVal.setException(new ServiceNotAvailableException(serviceName)); + } + + return; + } + + final Request request = requestBuilder.build(serviceLocation); + ListenableFuture> responseFuture; + + log.debug("Service [%s] request [%s %s] starting.", serviceName, request.getMethod(), request.getUrl()); + + responseFuture = httpClient.go( + request, + new ObjectOrErrorResponseHandler<>(handler), + requestBuilder.getTimeout() + ); + + // Add cancellation listener on the return future to ensure that responseFuture is canceled too. + final ListenableFuture> theResponseFuture = responseFuture; + + retVal.addListener( + () -> { + if (retVal.isCancelled()) { + theResponseFuture.cancel(true); + } + }, + Execs.directExecutor() + ); + + Futures.addCallback( + responseFuture, + new FutureCallback>() + { + @Override + public void onSuccess(@Nullable final Either result) + { + try { + // result can be null if the HttpClient encounters a problem midstream on an unfinished response. + if (result != null && result.isValue()) { + if (nextAttemptNumber > 1) { + // There were retries. Log at INFO level to provide the user some closure. + log.info( + "Service [%s] request [%s %s] completed.", + serviceName, + request.getMethod(), + request.getUrl() + ); + } else { + // No retries. Log at debug level to avoid cluttering the logs. + log.debug( + "Service [%s] request [%s %s] completed.", + serviceName, + request.getMethod(), + request.getUrl() + ); + } + + // Will not throw, because we checked result.isValue() earlier. + retVal.set(result.valueOrThrow()); + } else { + final StringFullResponseHolder errorHolder = result != null ? result.error() : null; + + if (errorHolder != null && isRedirect(errorHolder.getResponse().getStatus())) { + // Redirect. Update preferredLocationNoPath if appropriate, then reissue. + final String newUri = result.error().getResponse().headers().get("Location"); + + if (redirectCount >= MAX_REDIRECTS) { + retVal.setException(new RpcException("Service [%s] issued too many redirects", serviceName)); + } else { + // Update preferredLocationNoPath if we got a redirect. + final ServiceLocation redirectLocationNoPath = serviceLocationNoPathFromUri(newUri); + + if (redirectLocationNoPath != null) { + preferredLocationNoPath.set(redirectLocationNoPath); + connectExec.submit( + () -> tryRequest(requestBuilder, handler, retVal, attemptNumber, redirectCount + 1) + ); + } else { + retVal.setException( + new RpcException("Service [%s] redirected to invalid URL [%s]", serviceName, newUri) + ); + } + } + } else if (shouldTry(nextAttemptNumber) + && (errorHolder == null || retryPolicy.retryHttpResponse(errorHolder.getResponse()))) { + // Retryable server response (or null errorHolder, which means null result, which can happen + // if the HttpClient encounters an exception in the midst of response processing). + final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber); + log.noStackTrace().info(buildErrorMessage(request, errorHolder, backoffMs, nextAttemptNumber)); + connectExec.schedule( + () -> tryRequest(requestBuilder, handler, retVal, attemptNumber + 1, redirectCount), + backoffMs, + TimeUnit.MILLISECONDS + ); + } else if (errorHolder != null) { + // Nonretryable server response. + retVal.setException(new HttpResponseException(errorHolder)); + } else { + // Nonretryable null result from the HTTP client. + retVal.setException(new RpcException(buildErrorMessage(request, null, -1, nextAttemptNumber))); + } + } + } + catch (Throwable t) { + // It's a bug if this happens. The purpose of this line is to help us debug what went wrong. + retVal.setException(new RpcException(t, "Service [%s] handler exited unexpectedly", serviceName)); + } + } + + @Override + public void onFailure(final Throwable t) + { + try { + final long nextAttemptNumber = attemptNumber + 1; + + if (shouldTry(nextAttemptNumber) && retryPolicy.retryThrowable(t)) { + final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber); + + log.noStackTrace().info(t, buildErrorMessage(request, null, backoffMs, nextAttemptNumber)); + + connectExec.schedule( + () -> tryRequest(requestBuilder, handler, retVal, attemptNumber + 1, redirectCount), + backoffMs, + TimeUnit.MILLISECONDS + ); + } else { + retVal.setException(new RpcException(t, buildErrorMessage(request, null, -1, nextAttemptNumber))); + } + } + catch (Throwable t2) { + // It's a bug if this happens. The purpose of this line is to help us debug what went wrong. + retVal.setException(new RpcException(t, "Service [%s] handler exited unexpectedly", serviceName)); + } + } + }, + connectExec + ); + }, + retVal + ); + } + + private void whenServiceReady(final Consumer callback, final SettableFuture retVal) + { + Futures.addCallback( + serviceLocator.locate(), + new FutureCallback() + { + @Override + public void onSuccess(final ServiceLocations locations) + { + if (locations.isClosed()) { + retVal.setException(new ServiceClosedException(serviceName)); + return; + } + + try { + final ServiceLocation location = pick(locations); + callback.accept(location); + } + catch (Throwable t) { + // It's a bug if this happens. The purpose of this line is to help us debug what went wrong. + retVal.setException(new RpcException(t, "Service [%s] handler exited unexpectedly", serviceName)); + } + } + + @Override + public void onFailure(Throwable t) + { + // Service locator exceptions are not recoverable. + retVal.setException(new RpcException(t, "Service [%s] locator encountered exception", serviceName)); + } + }, + connectExec + ); + } + + @Nullable + private ServiceLocation pick(final ServiceLocations locations) + { + final ServiceLocation preferred = preferredLocationNoPath.get(); + + if (preferred != null) { + // Preferred location is set. Use it if it's one of known locations. + for (final ServiceLocation location : locations.getLocations()) { + final ServiceLocation locationNoPath = + new ServiceLocation(location.getHost(), location.getPlaintextPort(), location.getTlsPort(), ""); + + if (locationNoPath.equals(preferred)) { + return location; + } + } + } + + // No preferred location, or, preferred location is not one of the known service locations. Go with the first one. + return Iterables.getFirst(locations.getLocations(), null); + } + + private boolean shouldTry(final long nextAttemptNumber) + { + return retryPolicy.maxAttempts() < 0 || nextAttemptNumber < retryPolicy.maxAttempts(); + } + + private String buildErrorMessage( + final Request request, + @Nullable final StringFullResponseHolder errorHolder, + final long backoffMs, + final long numAttempts + ) + { + final StringBuilder errorMessage = new StringBuilder(); + + errorMessage.append("Service [") + .append(serviceName) + .append("] request [") + .append(request.getMethod()) + .append(" ") + .append(request.getUrl()) + .append("]"); + + if (errorHolder != null) { + final HttpResponseStatus httpResponseStatus = errorHolder.getStatus(); + errorMessage.append(" encountered server error [").append(httpResponseStatus).append("]"); + } else { + errorMessage.append(" encountered exception"); + } + + errorMessage.append(" on attempt #").append(numAttempts); + + if (backoffMs > 0) { + errorMessage.append("; retrying in ").append(StringUtils.format("%,d", backoffMs)).append(" ms"); + } + + if (errorHolder != null) { + errorMessage.append("; ").append(HttpResponseException.choppedBodyErrorMessage(errorHolder.getContent())); + } + + return errorMessage.toString(); + } + + @VisibleForTesting + static long computeBackoffMs(final ServiceRetryPolicy retryPolicy, final long attemptNumber) + { + return Math.max( + retryPolicy.minWaitMillis(), + Math.min(retryPolicy.maxWaitMillis(), (long) (Math.pow(2, attemptNumber) * retryPolicy.minWaitMillis())) + ); + } + + @Nullable + @VisibleForTesting + static ServiceLocation serviceLocationNoPathFromUri(@Nullable final String uriString) + { + if (uriString == null) { + return null; + } + + try { + final URI uri = new URI(uriString); + final String host = uri.getHost(); + + if (host == null) { + return null; + } + + final String scheme = uri.getScheme(); + + if ("http".equals(scheme)) { + return new ServiceLocation(host, uri.getPort() < 0 ? 80 : uri.getPort(), -1, ""); + } else if ("https".equals(scheme)) { + return new ServiceLocation(host, -1, uri.getPort() < 0 ? 443 : uri.getPort(), ""); + } else { + return null; + } + } + catch (URISyntaxException e) { + return null; + } + } + + @VisibleForTesting + static boolean isRedirect(final HttpResponseStatus responseStatus) + { + final int code = responseStatus.getCode(); + return code == HttpResponseStatus.TEMPORARY_REDIRECT.getCode() + || code == HttpResponseStatus.FOUND.getCode() + || code == HttpResponseStatus.MOVED_PERMANENTLY.getCode(); + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClosedException.java b/server/src/main/java/org/apache/druid/rpc/ServiceClosedException.java new file mode 100644 index 00000000000..1189f4476f8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClosedException.java @@ -0,0 +1,31 @@ +/* + * 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; + +/** + * Returned by {@link ServiceClient#asyncRequest} when a request has failed because the service is closed. + */ +public class ServiceClosedException extends RpcException +{ + public ServiceClosedException(final String serviceName) + { + super("Service [%s] is closed", serviceName); + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java new file mode 100644 index 00000000000..eab82df328a --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java @@ -0,0 +1,102 @@ +/* + * 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.base.Preconditions; +import org.apache.druid.server.DruidNode; + +import java.util.Objects; + +/** + * Represents a service location at a particular point in time. + */ +public class ServiceLocation +{ + private final String host; + private final int plaintextPort; + private final int tlsPort; + private final String basePath; + + public ServiceLocation(final String host, final int plaintextPort, final int tlsPort, final String basePath) + { + this.host = Preconditions.checkNotNull(host, "host"); + this.plaintextPort = plaintextPort; + this.tlsPort = tlsPort; + this.basePath = Preconditions.checkNotNull(basePath, "basePath"); + } + + public static ServiceLocation fromDruidNode(final DruidNode druidNode) + { + return new ServiceLocation(druidNode.getHost(), druidNode.getPlaintextPort(), druidNode.getTlsPort(), ""); + } + + public String getHost() + { + return host; + } + + public int getPlaintextPort() + { + return plaintextPort; + } + + public int getTlsPort() + { + return tlsPort; + } + + public String getBasePath() + { + return basePath; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ServiceLocation that = (ServiceLocation) o; + return plaintextPort == that.plaintextPort + && tlsPort == that.tlsPort + && Objects.equals(host, that.host) + && Objects.equals(basePath, that.basePath); + } + + @Override + public int hashCode() + { + return Objects.hash(host, plaintextPort, tlsPort, basePath); + } + + @Override + public String toString() + { + return "ServiceLocation{" + + "host='" + host + '\'' + + ", plaintextPort=" + plaintextPort + + ", tlsPort=" + tlsPort + + ", basePath='" + basePath + '\'' + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceLocations.java b/server/src/main/java/org/apache/druid/rpc/ServiceLocations.java new file mode 100644 index 00000000000..f2080deee72 --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceLocations.java @@ -0,0 +1,99 @@ +/* + * 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.base.Preconditions; +import org.apache.druid.java.util.common.IAE; + +import java.util.Collections; +import java.util.Objects; +import java.util.Set; + +/** + * Returned by {@link ServiceLocator#locate()}. See that function for documentation. + */ +public class ServiceLocations +{ + private final Set locations; + private final boolean closed; + + private ServiceLocations(final Set locations, final boolean closed) + { + this.locations = Preconditions.checkNotNull(locations, "locations"); + this.closed = closed; + + if (closed && !locations.isEmpty()) { + throw new IAE("Locations must be empty for closed services"); + } + } + + public static ServiceLocations forLocation(final ServiceLocation location) + { + return new ServiceLocations(Collections.singleton(Preconditions.checkNotNull(location)), false); + } + + public static ServiceLocations forLocations(final Set locations) + { + return new ServiceLocations(locations, false); + } + + public static ServiceLocations closed() + { + return new ServiceLocations(Collections.emptySet(), true); + } + + public Set getLocations() + { + return locations; + } + + public boolean isClosed() + { + return closed; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ServiceLocations that = (ServiceLocations) o; + return closed == that.closed && Objects.equals(locations, that.locations); + } + + @Override + public int hashCode() + { + return Objects.hash(locations, closed); + } + + @Override + public String toString() + { + return "ServiceLocations{" + + "locations=" + locations + + ", closed=" + closed + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/ServiceLocator.java new file mode 100644 index 00000000000..e8991e66fda --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceLocator.java @@ -0,0 +1,45 @@ +/* + * 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.ListenableFuture; + +import java.io.Closeable; + +/** + * Used by {@link ServiceClient} to locate services. Thread-safe. + */ +public interface ServiceLocator extends Closeable +{ + /** + * Returns a future that resolves to a set of {@link ServiceLocation}. + * + * If the returned object returns true from {@link ServiceLocations#isClosed()}, it means the service has closed + * permanently. Otherwise, any of the returned locations in {@link ServiceLocations#getLocations()} is a viable + * selection. + * + * It is possible for the list of locations to be empty. This means that the service is not currently available, + * but also has not been closed, so it may become available at some point in the future. + */ + ListenableFuture locate(); + + @Override + void close(); +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceNotAvailableException.java b/server/src/main/java/org/apache/druid/rpc/ServiceNotAvailableException.java new file mode 100644 index 00000000000..290fa2cf7ab --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceNotAvailableException.java @@ -0,0 +1,31 @@ +/* + * 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; + +/** + * Returned by {@link ServiceClient#asyncRequest} when a request has failed because the service is not available. + */ +public class ServiceNotAvailableException extends RpcException +{ + public ServiceNotAvailableException(final String serviceName) + { + super("Service [%s] is not available", 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 new file mode 100644 index 00000000000..786a7b1ab8e --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/ServiceRetryPolicy.java @@ -0,0 +1,56 @@ +/* + * 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 org.jboss.netty.handler.codec.http.HttpResponse; + +/** + * Used by {@link ServiceClient} to decide whether to retry requests. + */ +public interface ServiceRetryPolicy +{ + int UNLIMITED = -1; + + /** + * Returns the maximum number of desired attempts, or {@link #UNLIMITED} if unlimited. A value of 1 means no retries. + * Zero is invalid. + */ + long maxAttempts(); + + /** + * Returns the minimum wait time between retries. + */ + long minWaitMillis(); + + /** + * Returns the maximum wait time between retries. + */ + long maxWaitMillis(); + + /** + * Returns whether the given HTTP response can be retried. The response will have a non-2xx error code. + */ + boolean retryHttpResponse(HttpResponse response); + + /** + * Returns whether the given exception can be retried. + */ + boolean retryThrowable(Throwable t); +} diff --git a/server/src/main/java/org/apache/druid/rpc/StandardRetryPolicy.java b/server/src/main/java/org/apache/druid/rpc/StandardRetryPolicy.java new file mode 100644 index 00000000000..5cb4e74e47a --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/StandardRetryPolicy.java @@ -0,0 +1,141 @@ +/* + * 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 org.apache.druid.java.util.common.IAE; +import org.jboss.netty.channel.ChannelException; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import java.io.IOException; + +/** + * Retry policy configurable with a maximum number of attempts and min/max wait time. + * + * The policy retries on IOExceptions and ChannelExceptions, and on HTTP 500, 502, 503, and 504. Other exceptions + * and other HTTP status codes are considered nonretryable errors. + */ +public class StandardRetryPolicy implements ServiceRetryPolicy +{ + private static final long DEFAULT_MIN_WAIT_MS = 100; + private static final long DEFAULT_MAX_WAIT_MS = 30_000; + + private static final StandardRetryPolicy DEFAULT_UNLIMITED_POLICY = new Builder().maxAttempts(UNLIMITED).build(); + private static final StandardRetryPolicy DEFAULT_NO_RETRIES_POLICY = new Builder().maxAttempts(1).build(); + + private final long maxAttempts; + private final long minWaitMillis; + private final long maxWaitMillis; + + private StandardRetryPolicy(long maxAttempts, long minWaitMillis, long maxWaitMillis) + { + this.maxAttempts = maxAttempts; + this.minWaitMillis = minWaitMillis; + this.maxWaitMillis = maxWaitMillis; + + if (maxAttempts == 0) { + throw new IAE("maxAttempts must be positive (limited) or negative (unlimited); cannot be zero."); + } + } + + public static Builder builder() + { + return new Builder(); + } + + public static StandardRetryPolicy unlimited() + { + return DEFAULT_UNLIMITED_POLICY; + } + + public static StandardRetryPolicy noRetries() + { + return DEFAULT_NO_RETRIES_POLICY; + } + + @Override + public long maxAttempts() + { + return maxAttempts; + } + + @Override + public long minWaitMillis() + { + return minWaitMillis; + } + + @Override + public long maxWaitMillis() + { + return maxWaitMillis; + } + + @Override + public boolean retryHttpResponse(final HttpResponse response) + { + final int code = response.getStatus().getCode(); + + return code == HttpResponseStatus.BAD_GATEWAY.getCode() + || code == HttpResponseStatus.SERVICE_UNAVAILABLE.getCode() + || code == HttpResponseStatus.GATEWAY_TIMEOUT.getCode() + + // Technically shouldn't retry this last one, but servers sometimes return HTTP 500 for retryable errors. + || code == HttpResponseStatus.INTERNAL_SERVER_ERROR.getCode(); + } + + @Override + public boolean retryThrowable(Throwable t) + { + return t instanceof IOException + || t instanceof ChannelException + || (t.getCause() != null && retryThrowable(t.getCause())); + } + + 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; + + public Builder maxAttempts(final long maxAttempts) + { + this.maxAttempts = maxAttempts; + return this; + } + + public Builder minWaitMillis(final long minWaitMillis) + { + this.minWaitMillis = minWaitMillis; + return this; + } + + public Builder maxWaitMillis(final long maxWaitMillis) + { + this.maxWaitMillis = maxWaitMillis; + return this; + } + + public StandardRetryPolicy build() + { + return new StandardRetryPolicy(maxAttempts, minWaitMillis, maxWaitMillis); + } + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/guice/ServiceClientModule.java b/server/src/main/java/org/apache/druid/rpc/guice/ServiceClientModule.java new file mode 100644 index 00000000000..1f7bb3b654e --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/guice/ServiceClientModule.java @@ -0,0 +1,99 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Provides; +import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.rpc.DiscoveryServiceLocator; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceClientFactoryImpl; +import org.apache.druid.rpc.ServiceLocator; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.rpc.indexing.OverlordClientImpl; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ScheduledExecutorService; + +public class ServiceClientModule implements DruidModule +{ + private static final int CONNECT_EXEC_THREADS = 4; + private static final int OVERLORD_ATTEMPTS = 6; + + @Override + public List getJacksonModules() + { + return Collections.emptyList(); + } + + @Override + public void configure(Binder binder) + { + // Nothing to do. + } + + @Provides + @LazySingleton + @EscalatedGlobal + public ServiceClientFactory makeServiceClientFactory(@EscalatedGlobal final HttpClient httpClient) + { + final ScheduledExecutorService connectExec = + ScheduledExecutors.fixed(CONNECT_EXEC_THREADS, "ServiceClientFactory-%d"); + return new ServiceClientFactoryImpl(httpClient, connectExec); + } + + @Provides + @ManageLifecycle + @IndexingService + public ServiceLocator makeOverlordServiceLocator(final DruidNodeDiscoveryProvider discoveryProvider) + { + return new DiscoveryServiceLocator(discoveryProvider, NodeRole.OVERLORD); + } + + @Provides + public OverlordClient makeOverlordClient( + @Json final ObjectMapper jsonMapper, + @EscalatedGlobal final ServiceClientFactory clientFactory, + @IndexingService final ServiceLocator serviceLocator + ) + { + return new OverlordClientImpl( + clientFactory.makeClient( + NodeRole.OVERLORD.getJsonName(), + serviceLocator, + StandardRetryPolicy.builder().maxAttempts(OVERLORD_ATTEMPTS).build() + ), + jsonMapper + ); + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java new file mode 100644 index 00000000000..9429c120b4f --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -0,0 +1,56 @@ +/* + * 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.indexing; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.TaskStatus; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.rpc.ServiceRetryPolicy; + +import java.util.Map; +import java.util.Set; + +/** + * High-level Overlord client. + * + * Similar to {@link org.apache.druid.client.indexing.IndexingServiceClient}, but backed by + * {@link org.apache.druid.rpc.ServiceClient} instead of {@link org.apache.druid.discovery.DruidLeaderClient}. + * + * All methods return futures, enabling asynchronous logic. If you want a synchronous response, use + * {@code FutureUtils.get} or {@code FutureUtils.getUnchecked}. + * + * Futures resolve to exceptions in the manner described by {@link org.apache.druid.rpc.ServiceClient#asyncRequest}. + * + * Typically acquired via Guice, where it is registered using {@link org.apache.druid.rpc.guice.ServiceClientModule}. + */ +public interface OverlordClient +{ + ListenableFuture runTask(String taskId, Object taskObject); + + ListenableFuture cancelTask(String taskId); + + ListenableFuture> taskStatuses(Set taskIds); + + ListenableFuture taskStatus(String taskId); + + ListenableFuture> taskReportAsMap(String taskId); + + OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy); +} diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java new file mode 100644 index 00000000000..5be78f92a63 --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -0,0 +1,159 @@ +/* + * 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.indexing; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.TaskStatus; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder; +import org.apache.druid.rpc.IgnoreHttpResponseHandler; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.jboss.netty.handler.codec.http.HttpMethod; + +import java.io.IOException; +import java.util.Map; +import java.util.Set; + +/** + * Production implementation of {@link OverlordClient}. + */ +public class OverlordClientImpl implements OverlordClient +{ + private final ServiceClient client; + private final ObjectMapper jsonMapper; + + public OverlordClientImpl(final ServiceClient client, final ObjectMapper jsonMapper) + { + this.client = Preconditions.checkNotNull(client, "client"); + this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); + } + + @Override + public ListenableFuture runTask(final String taskId, final Object taskObject) + { + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/task") + .jsonContent(jsonMapper, taskObject), + new BytesFullResponseHandler() + ), + holder -> { + final Map map = deserialize(holder, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); + final String returnedTaskId = (String) map.get("task"); + + Preconditions.checkState( + taskId.equals(returnedTaskId), + "Got a different taskId[%s]. Expected taskId[%s]", + returnedTaskId, + taskId + ); + + return null; + } + ); + } + + @Override + public ListenableFuture cancelTask(final String taskId) + { + final String path = StringUtils.format("/druid/indexer/v1/task/%s/shutdown", StringUtils.urlEncode(taskId)); + + return client.asyncRequest( + new RequestBuilder(HttpMethod.POST, path), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + @Override + public ListenableFuture> taskStatuses(final Set taskIds) + { + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/taskStatus") + .jsonContent(jsonMapper, taskIds), + new BytesFullResponseHandler() + ), + holder -> deserialize(holder, new TypeReference>() {}) + ); + } + + @Override + public ListenableFuture taskStatus(final String taskId) + { + final String path = StringUtils.format("/druid/indexer/v1/task/%s/status", StringUtils.urlEncode(taskId)); + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> deserialize(holder, TaskStatusResponse.class) + ); + } + + @Override + public ListenableFuture> taskReportAsMap(String taskId) + { + final String path = StringUtils.format("/druid/indexer/v1/task/%s/reports", StringUtils.urlEncode(taskId)); + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> deserialize(holder, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT) + ); + } + + @Override + public OverlordClientImpl withRetryPolicy(ServiceRetryPolicy retryPolicy) + { + return new OverlordClientImpl(client.withRetryPolicy(retryPolicy), jsonMapper); + } + + private T deserialize(final BytesFullResponseHolder bytesHolder, final Class clazz) + { + try { + return jsonMapper.readValue(bytesHolder.getContent(), clazz); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + private T deserialize(final BytesFullResponseHolder bytesHolder, final TypeReference typeReference) + { + try { + return jsonMapper.readValue(bytesHolder.getContent(), typeReference); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } +} 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 new file mode 100644 index 00000000000..fe88e582396 --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java @@ -0,0 +1,91 @@ +/* + * 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.indexing; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +/** + * Retry policy for tasks. Meant to be used together with {@link SpecificTaskServiceLocator}. + * + * Returns true from {@link #retryHttpResponse} when encountering an HTTP 400 or HTTP 404 with a + * {@link ChatHandlerResource#TASK_ID_HEADER} header for a different task. This can happen when a task is suspended and + * then later restored in a different location, and then some *other* task reuses its old port. This task-mismatch + * scenario is retried indefinitely, since we expect that the {@link SpecificTaskServiceLocator} will update the + * location at some point. + */ +public class SpecificTaskRetryPolicy implements ServiceRetryPolicy +{ + private final String taskId; + private final ServiceRetryPolicy baseRetryPolicy; + + public SpecificTaskRetryPolicy(final String taskId, final ServiceRetryPolicy baseRetryPolicy) + { + this.taskId = Preconditions.checkNotNull(taskId, "taskId"); + this.baseRetryPolicy = Preconditions.checkNotNull(baseRetryPolicy, "baseRetryPolicy"); + } + + @Override + public long maxAttempts() + { + return baseRetryPolicy.maxAttempts(); + } + + @Override + public long minWaitMillis() + { + return baseRetryPolicy.minWaitMillis(); + } + + @Override + public long maxWaitMillis() + { + return baseRetryPolicy.maxWaitMillis(); + } + + @Override + public boolean retryHttpResponse(final HttpResponse response) + { + return baseRetryPolicy.retryHttpResponse(response) || isTaskMismatch(response); + } + + @Override + public boolean retryThrowable(final Throwable t) + { + return StandardRetryPolicy.unlimited().retryThrowable(t); + } + + private boolean isTaskMismatch(final HttpResponse response) + { + // See class-level javadocs for details on why we do this. + if (response.getStatus().equals(HttpResponseStatus.NOT_FOUND) + || response.getStatus().equals(HttpResponseStatus.BAD_REQUEST)) { + final String headerTaskId = StringUtils.urlDecode(response.headers().get(ChatHandlerResource.TASK_ID_HEADER)); + return headerTaskId != null && !headerTaskId.equals(taskId); + } else { + return false; + } + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskServiceLocator.java new file mode 100644 index 00000000000..88f02e43090 --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskServiceLocator.java @@ -0,0 +1,198 @@ +/* + * 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.indexing; + +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 com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.rpc.ServiceLocation; +import org.apache.druid.rpc.ServiceLocations; +import org.apache.druid.rpc.ServiceLocator; + +import java.util.Collections; + +/** + * Service locator for a specific task. Uses the {@link OverlordClient#taskStatus} API to locate tasks. + * + * This locator has an internal cache that is updated if the last check has been over {@link #LOCATION_CACHE_MS} ago. + * + * This locator is Closeable, like all ServiceLocators, but it is not essential that you actually close it. Closing + * does not free any resources: it merely makes future calls to {@link #locate()} return + * {@link ServiceLocations#closed()}. + */ +public class SpecificTaskServiceLocator implements ServiceLocator +{ + private static final String BASE_PATH = "/druid/worker/v1/chat"; + private static final long LOCATION_CACHE_MS = 30_000; + + private final String taskId; + private final OverlordClient overlordClient; + private final Object lock = new Object(); + + @GuardedBy("lock") + private TaskState lastKnownState = TaskState.RUNNING; // Assume task starts out running. + + @GuardedBy("lock") + private ServiceLocation lastKnownLocation; + + @GuardedBy("lock") + private boolean closed = false; + + @GuardedBy("lock") + private long lastUpdateTime = -1; + + @GuardedBy("lock") + private SettableFuture pendingFuture = null; + + public SpecificTaskServiceLocator(final String taskId, final OverlordClient overlordClient) + { + this.taskId = taskId; + this.overlordClient = overlordClient; + } + + @Override + public ListenableFuture locate() + { + synchronized (lock) { + if (pendingFuture != null) { + return Futures.nonCancellationPropagating(pendingFuture); + } else if (closed || lastKnownState != TaskState.RUNNING) { + return Futures.immediateFuture(ServiceLocations.closed()); + } else if (lastKnownLocation == null || lastUpdateTime + LOCATION_CACHE_MS < System.currentTimeMillis()) { + final ListenableFuture taskStatusFuture; + + try { + taskStatusFuture = overlordClient.taskStatus(taskId); + } + catch (Exception e) { + throw new RuntimeException(e); + } + + // Use shared future for concurrent calls to "locate"; don't want multiple calls out to the Overlord at once. + // Alias pendingFuture to retVal in case taskStatusFuture is already resolved. (This will make the callback + // below execute immediately, firing and nulling out pendingFuture.) + final SettableFuture retVal = (pendingFuture = SettableFuture.create()); + pendingFuture.addListener( + () -> { + if (!taskStatusFuture.isDone()) { + // pendingFuture may resolve without taskStatusFuture due to close(). + taskStatusFuture.cancel(true); + } + }, + Execs.directExecutor() + ); + + Futures.addCallback( + taskStatusFuture, + new FutureCallback() + { + @Override + public void onSuccess(final TaskStatusResponse taskStatus) + { + synchronized (lock) { + if (pendingFuture != null) { + lastUpdateTime = System.currentTimeMillis(); + + final TaskStatusPlus statusPlus = taskStatus.getStatus(); + + if (statusPlus == null) { + // If the task status is unknown, we'll treat it as closed. + lastKnownState = null; + lastKnownLocation = null; + } else { + lastKnownState = statusPlus.getStatusCode(); + + if (TaskLocation.unknown().equals(statusPlus.getLocation())) { + lastKnownLocation = null; + } else { + lastKnownLocation = new ServiceLocation( + statusPlus.getLocation().getHost(), + statusPlus.getLocation().getPort(), + statusPlus.getLocation().getTlsPort(), + StringUtils.format("%s/%s", BASE_PATH, StringUtils.urlEncode(taskId)) + ); + } + } + + if (lastKnownState != TaskState.RUNNING) { + pendingFuture.set(ServiceLocations.closed()); + } else if (lastKnownLocation == null) { + pendingFuture.set(ServiceLocations.forLocations(Collections.emptySet())); + } else { + pendingFuture.set(ServiceLocations.forLocation(lastKnownLocation)); + } + + // Clear pendingFuture once it has been set. + pendingFuture = null; + } + } + } + + @Override + public void onFailure(Throwable t) + { + synchronized (lock) { + if (pendingFuture != null) { + pendingFuture.setException(t); + + // Clear pendingFuture once it has been set. + pendingFuture = null; + } + } + } + } + ); + + return Futures.nonCancellationPropagating(retVal); + } else { + return Futures.immediateFuture(ServiceLocations.forLocation(lastKnownLocation)); + } + } + } + + @Override + public void close() + { + // Class-level Javadocs promise that this method does not actually free resources: it only alters behavior + // for future calls to locate(). This is exploited in TaskServiceClients.makeClient. + + synchronized (lock) { + // Idempotent: can call close() multiple times so long as start() has already been called. + if (!closed) { + if (pendingFuture != null) { + pendingFuture.set(ServiceLocations.closed()); + + // Clear pendingFuture once it has been set. + pendingFuture = null; + } + + closed = true; + } + } + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/TaskServiceClients.java b/server/src/main/java/org/apache/druid/rpc/indexing/TaskServiceClients.java new file mode 100644 index 00000000000..54f56b8b4b9 --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/indexing/TaskServiceClients.java @@ -0,0 +1,53 @@ +/* + * 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.indexing; + +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceRetryPolicy; + +/** + * Utility function for creating {@link ServiceClient} instances that communicate to indexing service tasks. + */ +public class TaskServiceClients +{ + private TaskServiceClients() + { + // No instantiation. + } + + /** + * Makes a {@link ServiceClient} linked to the provided task. The client's base path comes pre-set to the + * chat handler resource of the task: {@code /druid/worker/v1/chat/}. + */ + public static ServiceClient makeClient( + final String taskId, + final ServiceRetryPolicy baseRetryPolicy, + final ServiceClientFactory serviceClientFactory, + final OverlordClient overlordClient + ) + { + // SpecificTaskServiceLocator is Closeable, but the close() method does not actually free resources, so there + // is no need to ensure it is ever closed. Relying on GC is OK. + final SpecificTaskServiceLocator serviceLocator = new SpecificTaskServiceLocator(taskId, overlordClient); + final SpecificTaskRetryPolicy retryPolicy = new SpecificTaskRetryPolicy(taskId, baseRetryPolicy); + return serviceClientFactory.makeClient(taskId, serviceLocator, retryPolicy); + } +} diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java deleted file mode 100644 index 447a32d3b40..00000000000 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java +++ /dev/null @@ -1,138 +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.client.indexing; - -import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.timeline.DataSegment; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class NoopIndexingServiceClient implements IndexingServiceClient -{ - @Override - public void killUnusedSegments(String idPrefix, String dataSource, Interval interval) - { - - } - - @Override - public int killPendingSegments(String dataSource, DateTime end) - { - return 0; - } - - @Override - public String compactSegments( - String idPrefix, - List segments, - int compactionTaskPriority, - @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, - @Nullable ClientCompactionTaskGranularitySpec granularitySpec, - @Nullable ClientCompactionTaskDimensionsSpec dimensionsSpec, - @Nullable AggregatorFactory[] metricsSpec, - @Nullable ClientCompactionTaskTransformSpec transformSpec, - @Nullable Boolean dropExisting, - @Nullable Map context - ) - { - return null; - } - - @Override - public int getTotalWorkerCapacity() - { - return 0; - } - - @Override - public int getTotalWorkerCapacityWithAutoScale() - { - return 0; - } - - @Override - public String runTask(String taskId, Object taskObject) - { - return null; - } - - @Override - public String cancelTask(String taskId) - { - return null; - } - - @Override - public List getActiveTasks() - { - return Collections.emptyList(); - } - - @Override - public TaskStatusResponse getTaskStatus(String taskId) - { - return new TaskStatusResponse(taskId, null); - } - - @Override - public Map getTaskStatuses(Set taskIds) - { - return Collections.emptyMap(); - } - - @Nullable - @Override - public TaskStatusPlus getLastCompleteTask() - { - return null; - } - - @Override - public TaskPayloadResponse getTaskPayload(String taskId) - { - return null; - } - - @Nullable - @Override - public Map getTaskReport(String taskId) - { - return null; - } - - @Override - public Map> getLockedIntervals(Map minTaskPriority) - { - return Collections.emptyMap(); - } - - @Override - public SamplerResponse sample(SamplerSpec samplerSpec) - { - return new SamplerResponse(0, 0, Collections.emptyList()); - } -} diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java new file mode 100644 index 00000000000..eb39ff65ece --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java @@ -0,0 +1,67 @@ +/* + * 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.client.indexing; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; + +import java.util.Map; +import java.util.Set; + +public class NoopOverlordClient implements OverlordClient +{ + @Override + public ListenableFuture runTask(String taskId, Object taskObject) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture cancelTask(String taskId) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture> taskStatuses(Set taskIds) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture taskStatus(String taskId) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture> taskReportAsMap(String taskId) + { + throw new UnsupportedOperationException(); + } + + @Override + public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) + { + // Ignore retryPolicy for the test client. + return this; + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/DiscoveryServiceLocatorTest.java b/server/src/test/java/org/apache/druid/rpc/DiscoveryServiceLocatorTest.java new file mode 100644 index 00000000000..48ae02708c7 --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/DiscoveryServiceLocatorTest.java @@ -0,0 +1,201 @@ +/* + * 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.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.server.DruidNode; +import org.junit.After; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.function.Consumer; + +public class DiscoveryServiceLocatorTest +{ + private static final DiscoveryDruidNode NODE1 = new DiscoveryDruidNode( + new DruidNode("test-service", "node1.example.com", false, -1, 8888, false, true), + NodeRole.BROKER, + Collections.emptyMap() + ); + + private static final DiscoveryDruidNode NODE2 = new DiscoveryDruidNode( + new DruidNode("test-service", "node2.example.com", false, -1, 8888, false, true), + NodeRole.BROKER, + Collections.emptyMap() + ); + + @Rule + public MockitoRule mockitoRule = MockitoJUnit.rule(); + + @Mock + public DruidNodeDiscoveryProvider discoveryProvider; + + private DiscoveryServiceLocator locator; + + @After + public void tearDown() + { + if (locator != null) { + locator.close(); + } + } + + @Test + public void test_locate_initializeEmpty() throws Exception + { + final TestDiscovery discovery = new TestDiscovery(); + Mockito.when(discoveryProvider.getForNodeRole(NodeRole.BROKER)).thenReturn(discovery); + locator = new DiscoveryServiceLocator(discoveryProvider, NodeRole.BROKER); + locator.start(); + + final ListenableFuture future = locator.locate(); + Assert.assertFalse(future.isDone()); + + discovery.fire(DruidNodeDiscovery.Listener::nodeViewInitialized); + Assert.assertEquals(ServiceLocations.forLocations(Collections.emptySet()), future.get()); + } + + @Test + public void test_locate_initializeNonEmpty() throws Exception + { + final TestDiscovery discovery = new TestDiscovery(); + Mockito.when(discoveryProvider.getForNodeRole(NodeRole.BROKER)).thenReturn(discovery); + locator = new DiscoveryServiceLocator(discoveryProvider, NodeRole.BROKER); + locator.start(); + + final ListenableFuture future = locator.locate(); + Assert.assertFalse(future.isDone()); + + discovery.fire(listener -> { + listener.nodesAdded(ImmutableSet.of(NODE1)); + listener.nodesAdded(ImmutableSet.of(NODE2)); + listener.nodeViewInitialized(); + }); + + Assert.assertEquals( + ServiceLocations.forLocations( + ImmutableSet.of( + ServiceLocation.fromDruidNode(NODE1.getDruidNode()), + ServiceLocation.fromDruidNode(NODE2.getDruidNode()) + ) + ), + future.get() + ); + } + + @Test + public void test_locate_removeAfterAdd() throws Exception + { + final TestDiscovery discovery = new TestDiscovery(); + Mockito.when(discoveryProvider.getForNodeRole(NodeRole.BROKER)).thenReturn(discovery); + locator = new DiscoveryServiceLocator(discoveryProvider, NodeRole.BROKER); + locator.start(); + + discovery.fire(listener -> { + listener.nodesAdded(ImmutableSet.of(NODE1)); + listener.nodesAdded(ImmutableSet.of(NODE2)); + listener.nodeViewInitialized(); + listener.nodesRemoved(ImmutableSet.of(NODE1)); + }); + + Assert.assertEquals( + ServiceLocations.forLocations( + ImmutableSet.of( + ServiceLocation.fromDruidNode(NODE2.getDruidNode()) + ) + ), + locator.locate().get() + ); + } + + @Test + public void test_locate_closed() throws Exception + { + final TestDiscovery discovery = new TestDiscovery(); + Mockito.when(discoveryProvider.getForNodeRole(NodeRole.BROKER)).thenReturn(discovery); + locator = new DiscoveryServiceLocator(discoveryProvider, NodeRole.BROKER); + locator.start(); + + final ListenableFuture future = locator.locate(); + locator.close(); + + Assert.assertEquals(ServiceLocations.closed(), future.get()); // Call made prior to close() + Assert.assertEquals(ServiceLocations.closed(), locator.locate().get()); // Call made after close() + + Assert.assertEquals(0, discovery.getListeners().size()); + } + + private static class TestDiscovery implements DruidNodeDiscovery + { + @GuardedBy("this") + private final List listeners; + + public TestDiscovery() + { + listeners = new ArrayList<>(); + } + + @Override + public Collection getAllNodes() + { + throw new UnsupportedOperationException(); + } + + @Override + public synchronized void registerListener(Listener listener) + { + listeners.add(listener); + } + + @Override + public synchronized void removeListener(Listener listener) + { + listeners.remove(listener); + } + + public synchronized List getListeners() + { + return ImmutableList.copyOf(listeners); + } + + public synchronized void fire(Consumer f) + { + for (final Listener listener : listeners) { + f.accept(listener); + } + } + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/NoDelayScheduledExecutorService.java b/server/src/test/java/org/apache/druid/rpc/NoDelayScheduledExecutorService.java new file mode 100644 index 00000000000..7574be33043 --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/NoDelayScheduledExecutorService.java @@ -0,0 +1,127 @@ +/* + * 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.ForwardingExecutorService; + +import java.util.concurrent.Callable; +import java.util.concurrent.Delayed; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * Used by {@link ServiceClientImplTest} so retries happen immediately. + */ +public class NoDelayScheduledExecutorService extends ForwardingExecutorService implements ScheduledExecutorService +{ + private final ExecutorService delegate; + + public NoDelayScheduledExecutorService(final ExecutorService delegate) + { + this.delegate = delegate; + } + + @Override + protected ExecutorService delegate() + { + return delegate; + } + + @Override + public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) + { + return new NoDelayScheduledFuture<>(delegate.submit(command)); + } + + @Override + public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) + { + return new NoDelayScheduledFuture<>(delegate.submit(callable)); + } + + @Override + public ScheduledFuture scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) + { + throw new UnsupportedOperationException(); + } + + @Override + public ScheduledFuture scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) + { + throw new UnsupportedOperationException(); + } + + private static class NoDelayScheduledFuture implements ScheduledFuture + { + private final Future delegate; + + public NoDelayScheduledFuture(final Future delegate) + { + this.delegate = delegate; + } + + @Override + public long getDelay(TimeUnit unit) + { + return 0; + } + + @Override + public int compareTo(Delayed o) + { + return 0; + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) + { + return delegate.cancel(mayInterruptIfRunning); + } + + @Override + public boolean isCancelled() + { + return delegate.isCancelled(); + } + + @Override + public boolean isDone() + { + return delegate.isDone(); + } + + @Override + public T get() throws InterruptedException, ExecutionException + { + return delegate.get(); + } + + @Override + public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException + { + return delegate.get(timeout, unit); + } + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/RequestBuilderTest.java b/server/src/test/java/org/apache/druid/rpc/RequestBuilderTest.java new file mode 100644 index 00000000000..ca95284ac56 --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/RequestBuilderTest.java @@ -0,0 +1,173 @@ +/* + * 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.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.io.ByteStreams; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.segment.TestHelper; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.jboss.netty.buffer.ChannelBufferInputStream; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.joda.time.Duration; +import org.junit.Assert; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import java.net.URI; + +public class RequestBuilderTest +{ + @Test + public void test_constructor_noLeadingSlash() + { + final IllegalArgumentException e = Assert.assertThrows( + IllegalArgumentException.class, + () -> new RequestBuilder(HttpMethod.GET, "q") + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Path must start with '/'")) + ); + } + + @Test + public void test_build_getPlaintext() throws Exception + { + final Request request = new RequestBuilder(HttpMethod.GET, "/q") + .header("x-test-header", "abc") + .header("x-test-header-2", "def") + .build(new ServiceLocation("example.com", 8888, -1, "")); + + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals(new URI("http://example.com:8888/q").toURL(), request.getUrl()); + Assert.assertEquals("abc", Iterables.getOnlyElement(request.getHeaders().get("x-test-header"))); + Assert.assertEquals("def", Iterables.getOnlyElement(request.getHeaders().get("x-test-header-2"))); + Assert.assertFalse(request.hasContent()); + } + + @Test + public void test_build_getTls() throws Exception + { + final Request request = new RequestBuilder(HttpMethod.GET, "/q") + .header("x-test-header", "abc") + .header("x-test-header-2", "def") + .build(new ServiceLocation("example.com", 9999, 8888, "")) /* TLS preferred over plaintext */; + + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals(new URI("https://example.com:8888/q").toURL(), request.getUrl()); + Assert.assertEquals("abc", Iterables.getOnlyElement(request.getHeaders().get("x-test-header"))); + Assert.assertEquals("def", Iterables.getOnlyElement(request.getHeaders().get("x-test-header-2"))); + Assert.assertFalse(request.hasContent()); + } + + @Test + public void test_build_getTlsWithBasePath() throws Exception + { + final Request request = new RequestBuilder(HttpMethod.GET, "/q") + .header("x-test-header", "abc") + .header("x-test-header-2", "def") + .build(new ServiceLocation("example.com", 9999, 8888, "/base")) /* TLS preferred over plaintext */; + + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals(new URI("https://example.com:8888/base/q").toURL(), request.getUrl()); + Assert.assertEquals("abc", Iterables.getOnlyElement(request.getHeaders().get("x-test-header"))); + Assert.assertEquals("def", Iterables.getOnlyElement(request.getHeaders().get("x-test-header-2"))); + Assert.assertFalse(request.hasContent()); + } + + @Test + public void test_build_postTlsNoContent() throws Exception + { + final Request request = new RequestBuilder(HttpMethod.POST, "/q") + .header("x-test-header", "abc") + .header("x-test-header-2", "def") + .build(new ServiceLocation("example.com", 9999, 8888, "")) /* TLS preferred over plaintext */; + + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals(new URI("https://example.com:8888/q").toURL(), request.getUrl()); + Assert.assertEquals("abc", Iterables.getOnlyElement(request.getHeaders().get("x-test-header"))); + Assert.assertEquals("def", Iterables.getOnlyElement(request.getHeaders().get("x-test-header-2"))); + Assert.assertFalse(request.hasContent()); + } + + @Test + public void test_build_postTlsWithContent() throws Exception + { + final String json = "{\"foo\": 3}"; + final Request request = new RequestBuilder(HttpMethod.POST, "/q") + .header("x-test-header", "abc") + .header("x-test-header-2", "def") + .content("application/json", StringUtils.toUtf8(json)) + .build(new ServiceLocation("example.com", 9999, 8888, "")) /* TLS preferred over plaintext */; + + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals(new URI("https://example.com:8888/q").toURL(), request.getUrl()); + Assert.assertEquals("abc", Iterables.getOnlyElement(request.getHeaders().get("x-test-header"))); + Assert.assertEquals("def", Iterables.getOnlyElement(request.getHeaders().get("x-test-header-2"))); + Assert.assertTrue(request.hasContent()); + + // Read and verify content. + Assert.assertEquals( + json, + StringUtils.fromUtf8(ByteStreams.toByteArray(new ChannelBufferInputStream(request.getContent()))) + ); + } + + @Test + public void test_build_postTlsWithJsonContent() throws Exception + { + final Request request = new RequestBuilder(HttpMethod.POST, "/q") + .header("x-test-header", "abc") + .header("x-test-header-2", "def") + .jsonContent(TestHelper.makeJsonMapper(), ImmutableMap.of("foo", 3)) + .build(new ServiceLocation("example.com", 9999, 8888, "")) /* TLS preferred over plaintext */; + + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals(new URI("https://example.com:8888/q").toURL(), request.getUrl()); + Assert.assertEquals("abc", Iterables.getOnlyElement(request.getHeaders().get("x-test-header"))); + Assert.assertEquals("def", Iterables.getOnlyElement(request.getHeaders().get("x-test-header-2"))); + Assert.assertTrue(request.hasContent()); + + // Read and verify content. + Assert.assertEquals( + "{\"foo\":3}", + StringUtils.fromUtf8(ByteStreams.toByteArray(new ChannelBufferInputStream(request.getContent()))) + ); + } + + @Test + public void test_timeout() + { + Assert.assertEquals(RequestBuilder.DEFAULT_TIMEOUT, new RequestBuilder(HttpMethod.GET, "/q").getTimeout()); + Assert.assertEquals( + Duration.standardSeconds(1), + new RequestBuilder(HttpMethod.GET, "/q").timeout(Duration.standardSeconds(1)).getTimeout() + ); + Assert.assertEquals( + Duration.ZERO, + new RequestBuilder(HttpMethod.GET, "/q").timeout(Duration.ZERO).getTimeout() + ); + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java new file mode 100644 index 00000000000..01540c5d7e1 --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java @@ -0,0 +1,636 @@ +/* + * 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.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +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; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.ObjectOrErrorResponseHandler; +import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.jboss.netty.buffer.ChannelBuffers; +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.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.mockito.ArgumentMatchers; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; +import org.mockito.stubbing.OngoingStubbing; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class ServiceClientImplTest +{ + private static final String SERVICE_NAME = "test-service"; + private static final ServiceLocation SERVER1 = new ServiceLocation("example.com", -1, 8888, "/q"); + private static final ServiceLocation SERVER2 = new ServiceLocation("example.com", -1, 9999, "/q"); + + private ScheduledExecutorService exec; + + @Rule + public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); + + @Mock + private HttpClient httpClient; + + @Mock + private ServiceLocator serviceLocator; + + private ServiceClient serviceClient; + + @Before + public void setUp() + { + exec = new NoDelayScheduledExecutorService(Execs.directExecutor()); + } + + @After + public void tearDown() throws Exception + { + exec.shutdownNow(); + + if (!exec.awaitTermination(30, TimeUnit.SECONDS)) { + throw new ISE("Unable to shutdown executor in time"); + } + } + + @Test + public void test_request_ok() throws Exception + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + final ImmutableMap expectedResponseObject = ImmutableMap.of("foo", "bar"); + + // OK response from SERVER1. + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1).thenReturn(valueResponse(expectedResponseObject)); + + serviceClient = makeServiceClient(StandardRetryPolicy.noRetries()); + final Map response = doRequest(serviceClient, requestBuilder); + + Assert.assertEquals(expectedResponseObject, response); + } + + @Test + public void test_request_serverError() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Error response from SERVER1. + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(errorResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, null, "oh no")); + + serviceClient = makeServiceClient(StandardRetryPolicy.builder().maxAttempts(2).build()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(HttpResponseException.class)); + + final HttpResponseException httpResponseException = (HttpResponseException) e.getCause(); + Assert.assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR, httpResponseException.getResponse().getStatus()); + Assert.assertEquals("oh no", httpResponseException.getResponse().getContent()); + } + + @Test + public void test_request_serverErrorRetry() throws Exception + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + final ImmutableMap expectedResponseObject = ImmutableMap.of("foo", "bar"); + + // Error response from SERVER1, then OK response. + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(errorResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, null, "oh no")) + .thenReturn(valueResponse(expectedResponseObject)); + + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + final Map response = doRequest(serviceClient, requestBuilder); + Assert.assertEquals(expectedResponseObject, response); + } + + @Test + public void test_request_ioError() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // IOException when contacting SERVER1. + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(Futures.immediateFailedFuture(new IOException("oh no"))); + + serviceClient = makeServiceClient(StandardRetryPolicy.builder().maxAttempts(2).build()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage( + CoreMatchers.containsString( + "Service [test-service] request [GET https://example.com:8888/q/foo] encountered exception on attempt #2" + ) + ) + ); + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); + MatcherAssert.assertThat(e.getCause().getCause(), CoreMatchers.instanceOf(IOException.class)); + MatcherAssert.assertThat( + e.getCause().getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("oh no")) + ); + } + + @Test + public void test_request_ioErrorRetry() throws Exception + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + final ImmutableMap expectedResponseObject = ImmutableMap.of("foo", "bar"); + + // IOException when contacting SERVER1, then OK response. + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(Futures.immediateFailedFuture(new IOException("oh no"))) + .thenReturn(valueResponse(expectedResponseObject)); + + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + final Map response = doRequest(serviceClient, requestBuilder); + + Assert.assertEquals(expectedResponseObject, response); + } + + @Test + public void test_request_nullResponseFromClient() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Null response when contacting SERVER1. (HttpClient does this if an exception is encountered during processing.) + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1).thenReturn(Futures.immediateFuture(null)); + + serviceClient = makeServiceClient(StandardRetryPolicy.builder().maxAttempts(2).build()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage( + CoreMatchers.containsString( + "Service [test-service] request [GET https://example.com:8888/q/foo] encountered exception on attempt #2" + ) + ) + ); + } + + @Test + public void test_request_nullResponseFromClientRetry() throws Exception + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + final ImmutableMap expectedResponseObject = ImmutableMap.of("foo", "bar"); + + // Null response when contacting SERVER1. (HttpClient does this if an exception is encountered during processing.) + // Then, OK response. + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(Futures.immediateFuture(null)) + .thenReturn(valueResponse(expectedResponseObject)); + + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + final Map response = doRequest(serviceClient, requestBuilder); + + Assert.assertEquals(expectedResponseObject, response); + } + + @Test + public void test_request_followRedirect() throws Exception + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + final ImmutableMap expectedResponseObject = ImmutableMap.of("foo", "bar"); + + // Redirect from SERVER1 -> SERVER2. + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(redirectResponse(requestBuilder.build(SERVER2).getUrl().toString())); + expectHttpCall(requestBuilder, SERVER2).thenReturn(valueResponse(expectedResponseObject)); + + serviceClient = makeServiceClient(StandardRetryPolicy.noRetries()); + final Map response = doRequest(serviceClient, requestBuilder); + + Assert.assertEquals(expectedResponseObject, response); + } + + @Test + public void test_request_tooManyRedirects() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Endless self-redirects. + stubLocatorCall(locations(SERVER1)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(redirectResponse(requestBuilder.build(SERVER1).getUrl().toString())); + + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("too many redirects")) + ); + } + + @Test + public void test_request_redirectInvalid() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Endless self-redirects. + stubLocatorCall(locations(SERVER1)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(redirectResponse("invalid-url")); + + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("redirected to invalid URL [invalid-url]")) + ); + } + + @Test + public void test_request_redirectNil() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Endless self-redirects. + stubLocatorCall(locations(SERVER1)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(errorResponse(HttpResponseStatus.TEMPORARY_REDIRECT, null, null)); + + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("redirected to invalid URL [null]")) + ); + } + + @Test + public void test_request_dontFollowRedirectToUnknownServer() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Redirect from SERVER1 -> SERVER2, but SERVER2 is unknown. + stubLocatorCall(locations(SERVER1)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(redirectResponse(requestBuilder.build(SERVER2).getUrl().toString())); + + serviceClient = makeServiceClient(StandardRetryPolicy.noRetries()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("too many redirects")) + ); + } + + @Test + public void test_request_serviceUnavailable() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Service unavailable. + stubLocatorCall(locations()); + + serviceClient = makeServiceClient(StandardRetryPolicy.noRetries()); + + 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_serviceUnavailableRetry() throws Exception + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + final ImmutableMap expectedResponseObject = ImmutableMap.of("foo", "bar"); + + // Service unavailable at first, then available. + Mockito.when(serviceLocator.locate()) + .thenReturn(Futures.immediateFuture(locations())) + .thenReturn(Futures.immediateFuture(locations(SERVER1))); + expectHttpCall(requestBuilder, SERVER1).thenReturn(valueResponse(expectedResponseObject)); + + serviceClient = makeServiceClient(StandardRetryPolicy.builder().maxAttempts(2).build()); + final Map response = doRequest(serviceClient, requestBuilder); + + Assert.assertEquals(expectedResponseObject, response); + } + + @Test + public void test_request_serviceClosed() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Closed service. + stubLocatorCall(ServiceLocations.closed()); + + // Closed services are not retryable. + // Use an unlimited retry policy to ensure that the future actually resolves. + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(ServiceClosedException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Service [test-service] is closed")) + ); + } + + @Test + public void test_request_serviceLocatorException() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Service locator returns a bad future. + stubLocatorCall(Futures.immediateFailedFuture(new ISE("oh no"))); + + // Service locator exceptions are not retryable. + // Use an unlimited retry policy to ensure that the future actually resolves. + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); + MatcherAssert.assertThat(e.getCause().getCause(), CoreMatchers.instanceOf(IllegalStateException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage( + CoreMatchers.containsString("Service [test-service] locator encountered exception") + ) + ); + MatcherAssert.assertThat( + e.getCause().getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("oh no")) + ); + } + + @Test + public void test_request_cancelBeforeServiceLocated() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Service that will never be located. + stubLocatorCall(SettableFuture.create()); + + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + + final ListenableFuture> response = doAsyncRequest(serviceClient, requestBuilder); + + Assert.assertTrue(response.cancel(true)); + Assert.assertTrue(response.isCancelled()); + } + + @Test + public void test_request_cancelDuringRetry() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Error response from SERVER1, then a stalled future that will never resolve. + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(errorResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, null, "oh no")) + .thenReturn(SettableFuture.create()); + + serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + final ListenableFuture> response = doAsyncRequest(serviceClient, requestBuilder); + + Assert.assertTrue(response.cancel(true)); + Assert.assertTrue(response.isCancelled()); + } + + @Test + public void test_computeBackoffMs() + { + final StandardRetryPolicy retryPolicy = StandardRetryPolicy.unlimited(); + + Assert.assertEquals(100, ServiceClientImpl.computeBackoffMs(retryPolicy, 0)); + Assert.assertEquals(200, ServiceClientImpl.computeBackoffMs(retryPolicy, 1)); + Assert.assertEquals(3200, ServiceClientImpl.computeBackoffMs(retryPolicy, 5)); + Assert.assertEquals(30000, ServiceClientImpl.computeBackoffMs(retryPolicy, 20)); + } + + @Test + public void test_serviceLocationNoPathFromUri() + { + Assert.assertNull(ServiceClientImpl.serviceLocationNoPathFromUri("/")); + + Assert.assertEquals( + new ServiceLocation("1.2.3.4", 9999, -1, ""), + ServiceClientImpl.serviceLocationNoPathFromUri("http://1.2.3.4:9999/foo") + ); + + Assert.assertEquals( + new ServiceLocation("1.2.3.4", 80, -1, ""), + ServiceClientImpl.serviceLocationNoPathFromUri("http://1.2.3.4/foo") + ); + + Assert.assertEquals( + new ServiceLocation("1.2.3.4", -1, 9999, ""), + ServiceClientImpl.serviceLocationNoPathFromUri("https://1.2.3.4:9999/foo") + ); + + Assert.assertEquals( + new ServiceLocation("1.2.3.4", -1, 443, ""), + ServiceClientImpl.serviceLocationNoPathFromUri("https://1.2.3.4/foo") + ); + } + + @Test + public void test_isRedirect() + { + Assert.assertTrue(ServiceClientImpl.isRedirect(HttpResponseStatus.FOUND)); + Assert.assertTrue(ServiceClientImpl.isRedirect(HttpResponseStatus.MOVED_PERMANENTLY)); + Assert.assertTrue(ServiceClientImpl.isRedirect(HttpResponseStatus.TEMPORARY_REDIRECT)); + Assert.assertFalse(ServiceClientImpl.isRedirect(HttpResponseStatus.OK)); + } + + private OngoingStubbing>> expectHttpCall( + final RequestBuilder requestBuilder, + final ServiceLocation location + ) + { + final Request expectedRequest = requestBuilder.build(location); + + return Mockito.when( + httpClient.go( + ArgumentMatchers.argThat( + request -> + request != null + && expectedRequest.getMethod().equals(request.getMethod()) + && expectedRequest.getUrl().equals(request.getUrl()) + ), + ArgumentMatchers.any(ObjectOrErrorResponseHandler.class), + ArgumentMatchers.eq(RequestBuilder.DEFAULT_TIMEOUT) + ) + ); + } + + private void stubLocatorCall(final ServiceLocations locations) + { + stubLocatorCall(Futures.immediateFuture(locations)); + } + + private void stubLocatorCall(final ListenableFuture locations) + { + Mockito.doReturn(locations).when(serviceLocator).locate(); + } + + private ServiceClient makeServiceClient(final ServiceRetryPolicy retryPolicy) + { + return new ServiceClientImpl(SERVICE_NAME, httpClient, serviceLocator, retryPolicy, exec); + } + + private static Map doRequest( + final ServiceClient serviceClient, + final RequestBuilder requestBuilder + ) throws InterruptedException, ExecutionException + { + return serviceClient.request(requestBuilder, null /* Not verified by mocks */); + } + + private static ListenableFuture> doAsyncRequest( + final ServiceClient serviceClient, + final RequestBuilder requestBuilder + ) + { + return serviceClient.asyncRequest(requestBuilder, null /* Not verified by mocks */); + } + + private static ListenableFuture> valueResponse(final T o) + { + return Futures.immediateFuture(Either.value(o)); + } + + private static ListenableFuture> errorResponse( + final HttpResponseStatus responseStatus, + @Nullable final Map headers, + @Nullable final String content + ) + { + final DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, responseStatus); + + if (headers != null) { + for (final Map.Entry headerEntry : headers.entrySet()) { + response.headers().add(headerEntry.getKey(), headerEntry.getValue()); + } + } + + if (content != null) { + response.setContent(ChannelBuffers.wrappedBuffer(ByteBuffer.wrap(StringUtils.toUtf8(content)))); + } + + final StringFullResponseHolder errorHolder = new StringFullResponseHolder(response, StandardCharsets.UTF_8); + return Futures.immediateFuture(Either.error(errorHolder)); + } + + private static ListenableFuture> redirectResponse(final String newLocation) + { + return errorResponse( + HttpResponseStatus.TEMPORARY_REDIRECT, + ImmutableMap.of("location", newLocation), + null + ); + } + + private static ServiceLocations locations(final ServiceLocation... locations) + { + // ImmutableSet retains order, which is important. + return ServiceLocations.forLocations(ImmutableSet.copyOf(locations)); + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java b/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java new file mode 100644 index 00000000000..3fba0c409e0 --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java @@ -0,0 +1,34 @@ +/* + * 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 nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class ServiceLocationTest +{ + @Test + public void test_equals() + { + EqualsVerifier.forClass(ServiceLocation.class) + .usingGetClass() + .verify(); + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/ServiceLocationsTest.java b/server/src/test/java/org/apache/druid/rpc/ServiceLocationsTest.java new file mode 100644 index 00000000000..811c4c9a909 --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/ServiceLocationsTest.java @@ -0,0 +1,69 @@ +/* + * 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.collect.ImmutableSet; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; + +public class ServiceLocationsTest +{ + @Test + public void test_forLocation() + { + final ServiceLocation location = new ServiceLocation("h", -1, 2, ""); + final ServiceLocations locations = ServiceLocations.forLocation(location); + + Assert.assertEquals(ImmutableSet.of(location), locations.getLocations()); + Assert.assertFalse(locations.isClosed()); + } + + @Test + public void test_forLocations() + { + final ServiceLocation location1 = new ServiceLocation("h", -1, 2, ""); + final ServiceLocation location2 = new ServiceLocation("h", -1, 2, ""); + + final ServiceLocations locations = ServiceLocations.forLocations(ImmutableSet.of(location1, location2)); + + Assert.assertEquals(ImmutableSet.of(location1, location2), locations.getLocations()); + Assert.assertFalse(locations.isClosed()); + } + + @Test + public void test_closed() + { + final ServiceLocations locations = ServiceLocations.closed(); + + Assert.assertEquals(Collections.emptySet(), locations.getLocations()); + Assert.assertTrue(locations.isClosed()); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(ServiceLocations.class) + .usingGetClass() + .verify(); + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/SpecificTaskServiceLocatorTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/SpecificTaskServiceLocatorTest.java new file mode 100644 index 00000000000..85ae025c734 --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/indexing/SpecificTaskServiceLocatorTest.java @@ -0,0 +1,172 @@ +/* + * 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.indexing; + +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.client.indexing.TaskStatusResponse; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.rpc.ServiceLocation; +import org.apache.druid.rpc.ServiceLocations; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; + +import java.util.Collections; +import java.util.concurrent.ExecutionException; + +public class SpecificTaskServiceLocatorTest +{ + private static final String TASK_ID = "test-task"; + private static final TaskLocation TASK_LOCATION1 = TaskLocation.create("example.com", -1, 9998); + private static final ServiceLocation SERVICE_LOCATION1 = + new ServiceLocation("example.com", -1, 9998, "/druid/worker/v1/chat/test-task"); + + @Rule + public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); + + @Mock + private OverlordClient overlordClient; + + @Test + public void test_locate_noLocationYet() throws Exception + { + Mockito.when(overlordClient.taskStatus(TASK_ID)) + .thenReturn(makeResponse(TaskState.RUNNING, TaskLocation.unknown())); + + final SpecificTaskServiceLocator locator = new SpecificTaskServiceLocator(TASK_ID, overlordClient); + final ListenableFuture future = locator.locate(); + Assert.assertEquals(ServiceLocations.forLocations(Collections.emptySet()), future.get()); + } + + @Test + public void test_locate_taskRunning() throws Exception + { + Mockito.when(overlordClient.taskStatus(TASK_ID)) + .thenReturn(makeResponse(TaskState.RUNNING, TASK_LOCATION1)); + + final SpecificTaskServiceLocator locator = new SpecificTaskServiceLocator(TASK_ID, overlordClient); + Assert.assertEquals(ServiceLocations.forLocation(SERVICE_LOCATION1), locator.locate().get()); + } + + @Test + public void test_locate_taskNotFound() throws Exception + { + Mockito.when(overlordClient.taskStatus(TASK_ID)) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(TASK_ID, null))); + + final SpecificTaskServiceLocator locator = new SpecificTaskServiceLocator(TASK_ID, overlordClient); + final ListenableFuture future = locator.locate(); + Assert.assertEquals(ServiceLocations.closed(), future.get()); + } + + @Test + public void test_locate_taskSuccess() throws Exception + { + Mockito.when(overlordClient.taskStatus(TASK_ID)) + .thenReturn(makeResponse(TaskState.SUCCESS, TaskLocation.unknown())); + + final SpecificTaskServiceLocator locator = new SpecificTaskServiceLocator(TASK_ID, overlordClient); + final ListenableFuture future = locator.locate(); + Assert.assertEquals(ServiceLocations.closed(), future.get()); + } + + @Test + public void test_locate_taskFailed() throws Exception + { + Mockito.when(overlordClient.taskStatus(TASK_ID)) + .thenReturn(makeResponse(TaskState.FAILED, TaskLocation.unknown())); + + final SpecificTaskServiceLocator locator = new SpecificTaskServiceLocator(TASK_ID, overlordClient); + final ListenableFuture future = locator.locate(); + Assert.assertEquals(ServiceLocations.closed(), future.get()); + } + + @Test + public void test_locate_overlordError() + { + Mockito.when(overlordClient.taskStatus(TASK_ID)) + .thenReturn(Futures.immediateFailedFuture(new ISE("oh no"))); + + final SpecificTaskServiceLocator locator = new SpecificTaskServiceLocator(TASK_ID, overlordClient); + final ListenableFuture future = locator.locate(); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + future::get + ); + + MatcherAssert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("oh no"))); + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IllegalStateException.class)); + } + + @Test + public void test_locate_afterClose() throws Exception + { + // Overlord call will never return. + final SettableFuture overlordFuture = SettableFuture.create(); + Mockito.when(overlordClient.taskStatus(TASK_ID)) + .thenReturn(overlordFuture); + + final SpecificTaskServiceLocator locator = new SpecificTaskServiceLocator(TASK_ID, overlordClient); + final ListenableFuture future = locator.locate(); + locator.close(); + + Assert.assertEquals(ServiceLocations.closed(), future.get()); // Call prior to close + Assert.assertEquals(ServiceLocations.closed(), locator.locate().get()); // Call after close + Assert.assertTrue(overlordFuture.isCancelled()); + } + + private static ListenableFuture makeResponse(final TaskState state, final TaskLocation location) + { + final TaskStatusResponse response = new TaskStatusResponse( + TASK_ID, + new TaskStatusPlus( + TASK_ID, + null, + null, + DateTimes.utc(0), + DateTimes.utc(0), + state, + null, + null, + 1L, + location, + null, + null + ) + ); + + return Futures.immediateFuture(response); + } +} diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index f423982b9f7..eb1fff4d1ab 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -27,12 +27,9 @@ import com.google.inject.Inject; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; -import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Names; import com.google.inject.util.Providers; -import org.apache.druid.client.indexing.HttpIndexingServiceClient; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.discovery.WorkerNodeService; @@ -51,8 +48,7 @@ import org.apache.druid.guice.PolyBind; import org.apache.druid.guice.annotations.Self; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.overlord.ForkingTaskRunner; import org.apache.druid.indexing.overlord.TaskRunner; @@ -140,9 +136,7 @@ public class CliMiddleManager extends ServerRunnable binder.bind(ForkingTaskRunner.class).in(ManageLifecycle.class); binder.bind(WorkerTaskCountStatsProvider.class).to(ForkingTaskRunner.class); - binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); - binder.bind(new TypeLiteral>() {}) - .toProvider(Providers.of(null)); + binder.bind(ParallelIndexSupervisorTaskClientProvider.class).toProvider(Providers.of(null)); binder.bind(ShuffleClient.class).toProvider(Providers.of(null)); binder.bind(ChatHandlerProvider.class).toProvider(Providers.of(new NoopChatHandlerProvider())); PolyBind.createChoice( diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index c08dac3aeeb..14ec99195bc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -35,9 +35,7 @@ import com.google.inject.name.Names; import com.google.inject.servlet.GuiceFilter; import com.google.inject.util.Providers; import org.apache.druid.audit.AuditManager; -import org.apache.druid.client.indexing.HttpIndexingServiceClient; import org.apache.druid.client.indexing.IndexingService; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.IndexingServiceSelectorConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.IndexingServiceFirehoseModule; @@ -61,8 +59,7 @@ import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import org.apache.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; @@ -213,10 +210,7 @@ public class CliOverlord extends ServerRunnable binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); binder.bind(SupervisorManager.class).in(LazySingleton.class); - binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); - binder.bind(new TypeLiteral>() - { - }).toProvider(Providers.of(null)); + binder.bind(ParallelIndexSupervisorTaskClientProvider.class).toProvider(Providers.of(null)); binder.bind(ShuffleClient.class).toProvider(Providers.of(null)); binder.bind(ChatHandlerProvider.class).toProvider(Providers.of(new NoopChatHandlerProvider())); diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 717d036cf63..f46eb414152 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -33,15 +33,12 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; -import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Named; import com.google.inject.name.Names; import io.netty.util.SuppressForbidden; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.client.indexing.HttpIndexingServiceClient; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Binders; @@ -77,12 +74,11 @@ import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.DeepStorageShuffleClient; import org.apache.druid.indexing.common.task.batch.parallel.HttpShuffleClient; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskClientFactory; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProviderImpl; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -430,10 +426,9 @@ public class CliPeon extends GuiceRunnable JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class); configureTaskActionClient(binder); - binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); - binder.bind(new TypeLiteral>(){}) - .to(ParallelIndexTaskClientFactory.class) + binder.bind(ParallelIndexSupervisorTaskClientProvider.class) + .to(ParallelIndexSupervisorTaskClientProviderImpl.class) .in(LazySingleton.class); binder.bind(RetryPolicyFactory.class).in(LazySingleton.class);