From 2f9619a96fbde224a7de3233d278087fd4782a0f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 24 Jul 2023 21:14:27 -0700 Subject: [PATCH] Use OverlordClient for all Overlord RPCs. (#14581) * Use OverlordClient for all Overlord RPCs. Continuing the work from #12696, this patch removes HttpIndexingServiceClient and the IndexingService flavor of DruidLeaderClient completely. All remaining usages are migrated to OverlordClient. Supporting changes include: 1) Add a variety of methods to OverlordClient. 2) Update MetadataTaskStorage to skip the complete-task lookup when the caller requests zero completed tasks. This helps performance of the "get active tasks" APIs, which don't want to see complete ones. * Use less forbidden APIs. * Fixes from CI. * Add test coverage. * Two more tests. * Fix test. * Updates from CR. * Remove unthrown exceptions. * Refactor to improve testability and test coverage. * Add isNil tests. * Remove unnecessary "deserialize" methods. --- .../apache/druid/msq/guice/SqlTaskModule.java | 4 - .../overlord/HeapMemoryTaskStorage.java | 41 +- .../overlord/MetadataTaskStorage.java | 36 +- .../indexing/overlord/TaskStorageUtils.java | 66 ++ .../overlord/http/OverlordResource.java | 4 +- .../indexing/worker/WorkerTaskManager.java | 167 +++-- .../indexing/worker/WorkerTaskMonitor.java | 5 +- .../druid/indexing/common/TestUtils.java | 3 +- .../overlord/HeapMemoryTaskStorageTest.java | 41 + .../overlord/TaskStorageUtilsTest.java | 93 +++ ...bleStreamIndexTaskClientAsyncImplTest.java | 78 +- .../worker/WorkerTaskManagerTest.java | 166 ++++- .../worker/WorkerTaskMonitorTest.java | 4 +- .../org/apache/druid/metadata/TaskLookup.java | 35 +- .../druid/common/guava/FutureUtilsTest.java | 28 + .../apache/druid/metadata/TaskLookupTest.java | 32 +- .../druid/client/JsonParserIterator.java | 5 +- .../indexing/ClientCompactionTaskQuery.java | 2 +- .../client/indexing/ClientTaskQuery.java | 3 +- .../indexing/HttpIndexingServiceClient.java | 472 ------------ .../indexing/IndexingServiceClient.java | 102 --- .../IndexingTotalWorkerCapacityInfo.java | 31 + .../druid/client/indexing/IndexingWorker.java | 36 + .../client/indexing/IndexingWorkerInfo.java | 50 +- .../guice/IndexingServiceDiscoveryModule.java | 58 -- .../initialization/CoreInjectorBuilder.java | 2 - .../druid/rpc/indexing/OverlordClient.java | 140 +++- .../rpc/indexing/OverlordClientImpl.java | 206 ++++- .../server/coordinator/DruidCoordinator.java | 12 +- .../coordinator/KillStalePendingSegments.java | 35 +- .../coordinator/duty/CompactSegments.java | 161 +++- .../coordinator/duty/KillUnusedSegments.java | 11 +- .../server/http/DataSourcesResource.java | 15 +- .../server/http/OverlordProxyServlet.java | 16 +- .../HttpIndexingServiceClientTest.java | 469 ------------ .../client/indexing/NoopOverlordClient.java | 55 +- .../indexing/IndexingWorkerInfoTest.java} | 20 +- .../druid/indexing/IndexingWorkerTest.java | 33 + .../apache/druid/rpc/MockServiceClient.java | 8 +- .../rpc/indexing/OverlordClientImplTest.java | 405 +++++++++- .../coordinator/duty/CompactSegmentsTest.java | 705 ++++++------------ .../duty/KillUnusedSegmentsTest.java | 24 +- .../server/http/DataSourcesResourceTest.java | 25 +- .../server/http/OverlordProxyServletTest.java | 18 +- .../org/apache/druid/cli/CliCoordinator.java | 4 - .../org/apache/druid/cli/CliOverlord.java | 6 +- .../sql/calcite/schema/SystemSchema.java | 99 +-- .../schema/DruidCalciteSchemaModuleTest.java | 3 + .../sql/calcite/schema/SystemSchemaTest.java | 141 ++-- .../druid/sql/calcite/util/CalciteTests.java | 29 +- .../apache/druid/sql/guice/SqlModuleTest.java | 3 + 51 files changed, 2111 insertions(+), 2096 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageUtils.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskStorageUtilsTest.java delete mode 100644 server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java delete mode 100644 server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java delete mode 100644 server/src/main/java/org/apache/druid/guice/IndexingServiceDiscoveryModule.java delete mode 100644 server/src/test/java/org/apache/druid/client/indexing/HttpIndexingServiceClientTest.java rename server/src/{main/java/org/apache/druid/client/indexing/IndexingServiceSelectorConfig.java => test/java/org/apache/druid/indexing/IndexingWorkerInfoTest.java} (70%) create mode 100644 server/src/test/java/org/apache/druid/indexing/IndexingWorkerTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java index bcd8abfbb87..52531294f34 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java @@ -21,11 +21,8 @@ package org.apache.druid.msq.guice; import com.fasterxml.jackson.databind.Module; import com.google.inject.Binder; -import org.apache.druid.client.indexing.HttpIndexingServiceClient; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Jerseys; -import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.annotations.LoadScope; import org.apache.druid.initialization.DruidModule; @@ -48,7 +45,6 @@ public class SqlTaskModule implements DruidModule LifecycleModule.register(binder, SqlTaskResource.class); Jerseys.addResource(binder, SqlTaskResource.class); LifecycleModule.register(binder, SqlStatementResource.class); - binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); Jerseys.addResource(binder, SqlStatementResource.class); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index 52285f1f3c3..d01b4b4a3f8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -184,10 +184,7 @@ public class HeapMemoryTaskStorage implements TaskStorage return listBuilder.build(); } - public List> getRecentlyCreatedAlreadyFinishedTaskInfo( - CompleteTaskLookup taskLookup, - @Nullable String datasource - ) + public List> getRecentlyCreatedAlreadyFinishedTaskInfo(CompleteTaskLookup taskLookup) { final Ordering createdDateDesc = new Ordering() { @@ -216,17 +213,15 @@ public class HeapMemoryTaskStorage implements TaskStorage ) { final List> tasks = new ArrayList<>(); - taskLookups.forEach((type, lookup) -> { - if (type == TaskLookupType.COMPLETE) { - CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) lookup; - tasks.addAll( - getRecentlyCreatedAlreadyFinishedTaskInfo( - completeTaskLookup.hasTaskCreatedTimeFilter() - ? completeTaskLookup - : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()), - datasource - ) + final Map processedTaskLookups = + TaskStorageUtils.processTaskLookups( + taskLookups, + DateTimes.nowUtc().minus(config.getRecentlyFinishedThreshold()) ); + + processedTaskLookups.forEach((type, lookup) -> { + if (type == TaskLookupType.COMPLETE) { + tasks.addAll(getRecentlyCreatedAlreadyFinishedTaskInfo((CompleteTaskLookup) lookup)); } else { tasks.addAll(getActiveTaskInfo(datasource)); } @@ -319,10 +314,10 @@ public class HeapMemoryTaskStorage implements TaskStorage // It is then possible that the same task will be queued for removal twice. Whilst not ideal, // it will not cause any problems. List taskIds = tasks.entrySet().stream() - .filter(entry -> entry.getValue().getStatus().isComplete() - && entry.getValue().getCreatedDate().isBefore(timestamp)) - .map(Map.Entry::getKey) - .collect(Collectors.toList()); + .filter(entry -> entry.getValue().getStatus().isComplete() + && entry.getValue().getCreatedDate().isBefore(timestamp)) + .map(Map.Entry::getKey) + .collect(Collectors.toList()); taskIds.forEach(tasks::remove); synchronized (taskActions) { @@ -395,11 +390,11 @@ public class HeapMemoryTaskStorage implements TaskStorage static TaskInfo toTaskInfo(TaskStuff taskStuff) { return new TaskInfo<>( - taskStuff.getTask().getId(), - taskStuff.getCreatedDate(), - taskStuff.getStatus(), - taskStuff.getDataSource(), - taskStuff.getTask() + taskStuff.getTask().getId(), + taskStuff.getCreatedDate(), + taskStuff.getStatus(), + taskStuff.getDataSource(), + taskStuff.getTask() ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 2735802b2d7..251a2c8ce31 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -24,7 +24,6 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; import com.google.inject.Inject; import org.apache.druid.common.exception.DruidException; import org.apache.druid.indexer.TaskInfo; @@ -47,7 +46,6 @@ import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; -import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; import javax.annotation.Nullable; @@ -228,7 +226,11 @@ public class MetadataTaskStorage implements TaskStorage @Nullable String datasource ) { - Map theTaskLookups = processTaskLookups(taskLookups); + Map theTaskLookups = + TaskStorageUtils.processTaskLookups( + taskLookups, + DateTimes.nowUtc().minus(config.getRecentlyFinishedThreshold()) + ); return Collections.unmodifiableList(handler.getTaskInfos(theTaskLookups, datasource)); } @@ -238,7 +240,12 @@ public class MetadataTaskStorage implements TaskStorage @Nullable String datasource ) { - Map processedTaskLookups = processTaskLookups(taskLookups); + Map processedTaskLookups = + TaskStorageUtils.processTaskLookups( + taskLookups, + DateTimes.nowUtc().minus(config.getRecentlyFinishedThreshold()) + ); + return Collections.unmodifiableList( handler.getTaskStatusList(processedTaskLookups, datasource) .stream() @@ -247,27 +254,6 @@ public class MetadataTaskStorage implements TaskStorage ); } - private Map processTaskLookups( - Map taskLookups - ) - { - Map processedTaskLookups = Maps.newHashMapWithExpectedSize(taskLookups.size()); - for (Entry entry : taskLookups.entrySet()) { - if (entry.getKey() == TaskLookupType.COMPLETE) { - CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - processedTaskLookups.put( - entry.getKey(), - completeTaskLookup.hasTaskCreatedTimeFilter() - ? completeTaskLookup - : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()) - ); - } else { - processedTaskLookups.put(entry.getKey(), entry.getValue()); - } - } - return processedTaskLookups; - } - @Override public void addLock(final String taskid, final TaskLock taskLock) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageUtils.java new file mode 100644 index 00000000000..0323b5c836d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageUtils.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord; + +import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.metadata.TaskLookup; +import org.joda.time.DateTime; + +import java.util.LinkedHashMap; +import java.util.Map; + +public class TaskStorageUtils +{ + private TaskStorageUtils() + { + // No instantiation. + } + + /** + * Process a map of {@link TaskLookup} to apply {@link TaskStorageConfig#getRecentlyFinishedThreshold()}, and to + * remove lookups for which {@link TaskLookup#isNil()}. + * + * @param taskLookups lookups from {@link TaskStorage#getTaskInfos(Map, String)} + * @param minCreationTimestamp minimum creation time based on {@link TaskStorageConfig#getRecentlyFinishedThreshold()} + */ + public static Map processTaskLookups( + final Map taskLookups, + final DateTime minCreationTimestamp + ) + { + final Map retVal = new LinkedHashMap<>(); + + for (Map.Entry entry : taskLookups.entrySet()) { + if (!entry.getValue().isNil()) { + if (entry.getKey() == TaskLookup.TaskLookupType.COMPLETE) { + TaskLookup.CompleteTaskLookup completeTaskLookup = (TaskLookup.CompleteTaskLookup) entry.getValue(); + retVal.put( + entry.getKey(), + completeTaskLookup.withMinTimestampIfAbsent(minCreationTimestamp) + ); + } else { + retVal.put(entry.getKey(), entry.getValue()); + } + } + } + + return retVal; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index c6de461f1ff..3742ecb8c3c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -138,7 +138,7 @@ public class OverlordResource private final AuthConfig authConfig; private AtomicReference workerConfigRef = null; - private static final List API_TASK_STATES = ImmutableList.of("pending", "waiting", "running", "complete"); + private static final List API_TASK_STATES = ImmutableList.of("pending", "waiting", "running", "complete"); private enum TaskStateLookup { @@ -185,7 +185,7 @@ public class OverlordResource } /** - * Warning, magic: {@link org.apache.druid.client.indexing.HttpIndexingServiceClient#runTask} may call this method + * Warning, magic: {@link org.apache.druid.rpc.indexing.OverlordClient#runTask} may call this method * remotely with {@link ClientTaskQuery} objects, but we deserialize {@link Task} objects. See the comment for {@link * ClientTaskQuery} for details. */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java index 0267ccad33b..06989d995d5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.worker; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; @@ -30,15 +29,15 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; -import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.concurrent.LifecycleLock; -import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; @@ -47,13 +46,12 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.apache.druid.rpc.HttpResponseException; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; -import org.jboss.netty.handler.codec.http.HttpHeaders; -import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import javax.ws.rs.core.MediaType; import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -63,6 +61,7 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -102,7 +101,7 @@ public class WorkerTaskManager private final AtomicBoolean disabled = new AtomicBoolean(false); - private final DruidLeaderClient overlordClient; + private final OverlordClient overlordClient; private final File storageDir; @Inject @@ -110,7 +109,7 @@ public class WorkerTaskManager ObjectMapper jsonMapper, TaskRunner taskRunner, TaskConfig taskConfig, - @IndexingService DruidLeaderClient overlordClient + OverlordClient overlordClient ) { this.jsonMapper = jsonMapper; @@ -439,7 +438,7 @@ public class WorkerTaskManager { synchronized (lock) { runningTasks.remove(taskId); - completedTasks.put(taskId, taskAnnouncement); + addCompletedTask(taskId, taskAnnouncement); try { FileUtils.writeAtomically( @@ -469,7 +468,7 @@ public class WorkerTaskManager String taskId = taskFile.getName(); TaskAnnouncement taskAnnouncement = jsonMapper.readValue(taskFile, TaskAnnouncement.class); if (taskId.equals(taskAnnouncement.getTaskId())) { - completedTasks.put(taskId, taskAnnouncement); + addCompletedTask(taskId, taskAnnouncement); } else { throw new ISE("Corrupted completed task on disk[%s].", taskFile.getAbsoluteFile()); } @@ -496,77 +495,7 @@ public class WorkerTaskManager completedTasksCleanupExecutor.scheduleAtFixedRate( () -> { try { - if (completedTasks.isEmpty()) { - log.debug("Skipping completed tasks cleanup. Its empty."); - return; - } - - ImmutableSet taskIds = ImmutableSet.copyOf(completedTasks.keySet()); - Map taskStatusesFromOverlord = null; - - try { - StringFullResponseHolder fullResponseHolder = overlordClient.go( - overlordClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/taskStatus") - .setContent(jsonMapper.writeValueAsBytes(taskIds)) - .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON) - .addHeader(HttpHeaders.Names.CONTENT_TYPE, MediaType.APPLICATION_JSON) - - ); - if (fullResponseHolder.getStatus().getCode() == 200) { - String responseContent = fullResponseHolder.getContent(); - taskStatusesFromOverlord = jsonMapper.readValue( - responseContent, - new TypeReference>() - { - } - ); - log.debug("Received completed task status response [%s].", responseContent); - } else if (fullResponseHolder.getStatus().getCode() == 404) { - // NOTE: this is to support backward compatibility, when overlord doesn't have "activeTasks" endpoint. - // this if clause should be removed in a future release. - log.debug("Deleting all completed tasks. Overlord appears to be running on older version."); - taskStatusesFromOverlord = ImmutableMap.of(); - } else { - log.info( - "Got non-success code[%s] from overlord while getting active tasks. will retry on next scheduled run.", - fullResponseHolder.getStatus().getCode() - ); - } - } - catch (Exception ex) { - log.warn(ex, "Exception while getting active tasks from overlord. will retry on next scheduled run."); - - if (ex instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - } - - if (taskStatusesFromOverlord == null) { - return; - } - - for (String taskId : taskIds) { - TaskStatus status = taskStatusesFromOverlord.get(taskId); - if (status == null || status.isComplete()) { - - log.debug( - "Deleting completed task[%s] information, overlord task status[%s].", - taskId, - status == null ? "unknown" : status.getStatusCode() - ); - - completedTasks.remove(taskId); - File taskFile = new File(getCompletedTaskDir(), taskId); - try { - Files.deleteIfExists(taskFile.toPath()); - changeHistory.addChangeRequest(new WorkerHistoryItem.TaskRemoval(taskId)); - } - catch (IOException ex) { - log.error(ex, "Failed to delete completed task from disk [%s].", taskFile); - } - - } - } + this.doCompletedTasksCleanup(); } catch (Throwable th) { log.error(th, "Got unknown exception while running the scheduled cleanup."); @@ -602,6 +531,80 @@ public class WorkerTaskManager return !disabled.get(); } + /** + * Remove items from {@link #completedTasks} that the Overlord believes has completed. Scheduled by + * {@link #scheduleCompletedTasksCleanup()}. + */ + void doCompletedTasksCleanup() throws InterruptedException + { + if (completedTasks.isEmpty()) { + log.debug("Skipping completed tasks cleanup, because there are no completed tasks."); + return; + } + + ImmutableSet taskIds = ImmutableSet.copyOf(completedTasks.keySet()); + Either> apiCallResult; + + try { + apiCallResult = Either.value(FutureUtils.get(overlordClient.taskStatuses(taskIds), true)); + log.debug("Received completed task status response [%s].", apiCallResult); + } + catch (ExecutionException e) { + if (e.getCause() instanceof HttpResponseException) { + final HttpResponseStatus status = ((HttpResponseException) e.getCause()).getResponse().getStatus(); + if (status.getCode() == 404) { + // NOTE: this is to support backward compatibility, when overlord doesn't have "activeTasks" endpoint. + // this if clause should be removed in a future release. + log.debug("Deleting all completed tasks. Overlord appears to be running on older version."); + apiCallResult = Either.value(ImmutableMap.of()); + } else { + apiCallResult = Either.error(e.getCause()); + } + } else { + apiCallResult = Either.error(e.getCause()); + } + } + + if (apiCallResult.isError()) { + log.warn( + apiCallResult.error(), + "Exception while getting active tasks from Overlord. Will retry on next scheduled run." + ); + + return; + } + + for (String taskId : taskIds) { + TaskStatus status = apiCallResult.valueOrThrow().get(taskId); + if (status == null || status.isComplete()) { + log.debug( + "Deleting completed task[%s] information, Overlord task status[%s].", + taskId, + status == null ? "unknown" : status.getStatusCode() + ); + + completedTasks.remove(taskId); + File taskFile = new File(getCompletedTaskDir(), taskId); + try { + Files.deleteIfExists(taskFile.toPath()); + changeHistory.addChangeRequest(new WorkerHistoryItem.TaskRemoval(taskId)); + } + catch (IOException ex) { + log.error(ex, "Failed to delete completed task from disk [%s].", taskFile); + } + } + } + } + + /** + * Add a completed task to {@link #completedTasks}. It will eventually be removed by + * {@link #doCompletedTasksCleanup()}. + */ + void addCompletedTask(final String taskId, final TaskAnnouncement taskAnnouncement) + { + completedTasks.put(taskId, taskAnnouncement); + } + private static class TaskDetails { private final Task task; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskMonitor.java index 27fb31e8d45..895bd29faf2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskMonitor.java @@ -26,9 +26,7 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; -import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.curator.CuratorUtils; -import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.config.TaskConfig; @@ -38,6 +36,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.rpc.indexing.OverlordClient; /** * This class is deprecated and required only to support {@link org.apache.druid.indexing.overlord.RemoteTaskRunner}. @@ -66,7 +65,7 @@ public class WorkerTaskMonitor extends WorkerTaskManager TaskConfig taskConfig, CuratorFramework cf, WorkerCuratorCoordinator workerCuratorCoordinator, - @IndexingService DruidLeaderClient overlordClient + OverlordClient overlordClient ) { super(jsonMapper, taskRunner, taskConfig, overlordClient); 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 44d629b2276..e8a6d9839a8 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 @@ -26,7 +26,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; 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.NoopOverlordClient; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.NoopInputSource; @@ -102,7 +101,7 @@ public class TestUtils .addValue(AuthorizerMapper.class, null) .addValue(RowIngestionMetersFactory.class, rowIngestionMetersFactory) .addValue(PruneSpecsHolder.class, PruneSpecsHolder.DEFAULT) - .addValue(IndexingServiceClient.class, OVERLORD_SERVICE_CLIENT) + .addValue(OverlordClient.class, OVERLORD_SERVICE_CLIENT) .addValue(AuthorizerMapper.class, new AuthorizerMapper(ImmutableMap.of())) .addValue(AppenderatorsManager.class, APPENDERATORS_MANAGER) .addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller()) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java index c3bcb877da1..12267a24f7c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java @@ -19,15 +19,21 @@ package org.apache.druid.indexing.overlord; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.metadata.TaskLookup; import org.joda.time.Period; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.util.List; + public class HeapMemoryTaskStorageTest { private HeapMemoryTaskStorage storage; @@ -54,4 +60,39 @@ public class HeapMemoryTaskStorageTest Assert.assertNull(storage.getTaskInfo(task1.getId())); Assert.assertNotNull(storage.getTaskInfo(task2.getId())); } + + @Test + public void testGetTaskInfos() + { + final NoopTask task1 = NoopTask.create("foo"); + final NoopTask task2 = NoopTask.create("bar"); + storage.insert(task1, TaskStatus.success(task1.getId())); + storage.insert(task2, TaskStatus.running(task2.getId())); + + // Active statuses + final List> taskInfosActive = storage.getTaskInfos( + ImmutableMap.of( + TaskLookup.TaskLookupType.ACTIVE, + TaskLookup.ActiveTaskLookup.getInstance(), + TaskLookup.TaskLookupType.COMPLETE, + new TaskLookup.CompleteTaskLookup(0, DateTimes.of("1970")) + ), + null + ); + + Assert.assertEquals(1, taskInfosActive.size()); + Assert.assertEquals(task2.getId(), taskInfosActive.get(0).getTask().getId()); + + // Complete statuses + final List> taskInfosComplete = storage.getTaskInfos( + ImmutableMap.of( + TaskLookup.TaskLookupType.COMPLETE, + new TaskLookup.CompleteTaskLookup(null, DateTimes.of("1970")) + ), + null + ); + + Assert.assertEquals(1, taskInfosComplete.size()); + Assert.assertEquals(task1.getId(), taskInfosComplete.get(0).getTask().getId()); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskStorageUtilsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskStorageUtilsTest.java new file mode 100644 index 00000000000..a696925fbce --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskStorageUtilsTest.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.metadata.TaskLookup; +import org.junit.Assert; +import org.junit.Test; + +public class TaskStorageUtilsTest +{ + @Test + public void test_processTaskLookups_defaultLimit_defaultTime() + { + Assert.assertEquals( + ImmutableMap.of( + TaskLookup.TaskLookupType.ACTIVE, + TaskLookup.ActiveTaskLookup.getInstance(), + TaskLookup.TaskLookupType.COMPLETE, + new TaskLookup.CompleteTaskLookup(null, DateTimes.of("2000")) + ), + TaskStorageUtils.processTaskLookups( + ImmutableMap.of( + TaskLookup.TaskLookupType.ACTIVE, + TaskLookup.ActiveTaskLookup.getInstance(), + TaskLookup.TaskLookupType.COMPLETE, + TaskLookup.CompleteTaskLookup.of(null, null) + ), + DateTimes.of("2000") + ) + ); + } + + @Test + public void test_processTaskLookups_zeroCompleteTasks_defaultTime() + { + Assert.assertEquals( + ImmutableMap.of( + TaskLookup.TaskLookupType.ACTIVE, + TaskLookup.ActiveTaskLookup.getInstance() + ), + TaskStorageUtils.processTaskLookups( + ImmutableMap.of( + TaskLookup.TaskLookupType.ACTIVE, + TaskLookup.ActiveTaskLookup.getInstance(), + TaskLookup.TaskLookupType.COMPLETE, + new TaskLookup.CompleteTaskLookup(0, DateTimes.of("2000")) + ), + DateTimes.of("2000") + ) + ); + } + + @Test + public void test_processTaskLookups_oneCompleteTask_3000() + { + Assert.assertEquals( + ImmutableMap.of( + TaskLookup.TaskLookupType.ACTIVE, + TaskLookup.ActiveTaskLookup.getInstance(), + TaskLookup.TaskLookupType.COMPLETE, + new TaskLookup.CompleteTaskLookup(1, DateTimes.of("3000")) + ), + TaskStorageUtils.processTaskLookups( + ImmutableMap.of( + TaskLookup.TaskLookupType.ACTIVE, + TaskLookup.ActiveTaskLookup.getInstance(), + TaskLookup.TaskLookupType.COMPLETE, + new TaskLookup.CompleteTaskLookup(1, DateTimes.of("3000")) + ), + DateTimes.of("2000") + ) + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java index 0fa4e25ae32..72222ade56d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java @@ -96,7 +96,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final Map> checkpoints = ImmutableMap.of(0, ImmutableMap.of(2, 3L)); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/checkpoints").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -111,7 +111,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final ImmutableMap offsets = ImmutableMap.of(2, 3L); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/offsets/current").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -126,7 +126,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final ImmutableMap offsets = ImmutableMap.of(2, 3L); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/offsets/end").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -139,7 +139,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_getEndOffsetsAsync_notAvailable() throws Exception { - serviceClient.expect( + serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.GET, "/offsets/end").timeout(httpTimeout), new ServiceNotAvailableException(TASK_ID) ); @@ -150,7 +150,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_stopAsync_publish_ok() throws Exception { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/stop?publish=true").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -163,7 +163,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_stopAsync_noPublish_ok() throws Exception { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -176,7 +176,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_stopAsync_noPublish_httpError() throws Exception { - serviceClient.expect( + serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout), new HttpResponseException( new StringFullResponseHolder( @@ -192,7 +192,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_stopAsync_noPublish_notAvailable() throws Exception { - serviceClient.expect( + serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout), new ServiceNotAvailableException(TASK_ID) ); @@ -203,7 +203,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_stopAsync_noPublish_closed() throws Exception { - serviceClient.expect( + serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout), new ServiceClosedException(TASK_ID) ); @@ -214,7 +214,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_stopAsync_noPublish_ioException() { - serviceClient.expect( + serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/stop").timeout(httpTimeout), new IOException() ); @@ -230,7 +230,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_resumeAsync_ok() throws Exception { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/resume").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -243,7 +243,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_resumeAsync_ioException() throws Exception { - serviceClient.expect( + serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/resume").timeout(httpTimeout), new IOException() ); @@ -256,7 +256,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final Map offsets = ImmutableMap.of(1, 3L); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/offsets/end?finish=false") .content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(offsets)) .timeout(httpTimeout), @@ -271,7 +271,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_setEndOffsetsAsync_ioException() throws Exception { - serviceClient.expect( + serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/resume").timeout(httpTimeout), new IOException() ); @@ -282,7 +282,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_getStatusAsync() throws Exception { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -295,7 +295,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_getStatusAsync_notAvailable() throws Exception { - serviceClient.expect( + serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), new ServiceNotAvailableException(TASK_ID) ); @@ -308,7 +308,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final DateTime startTime = DateTimes.of("2000"); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/time/start").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -321,7 +321,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_getStartTimeAsync_noContent() throws Exception { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/time/start").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -334,7 +334,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_getStartTimeAsync_notAvailable() throws Exception { - serviceClient.expect( + serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.GET, "/time/start").timeout(httpTimeout), new ServiceNotAvailableException(TASK_ID) ); @@ -347,7 +347,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final Map offsets = ImmutableMap.of(1, 3L); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -362,7 +362,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final Map offsets = ImmutableMap.of(1, 3L); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout), HttpResponseStatus.CONTINUE, Collections.emptyMap(), @@ -386,17 +386,17 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final Map offsets = ImmutableMap.of(1, 3L); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout), HttpResponseStatus.ACCEPTED, Collections.emptyMap(), ByteArrays.EMPTY_ARRAY - ).expect( + ).expectAndRespond( new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.PAUSED) - ).expect( + ).expectAndRespond( new RequestBuilder(HttpMethod.GET, "/offsets/current").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -409,12 +409,12 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_pauseAsync_oneIterationWithError() { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout), HttpResponseStatus.ACCEPTED, Collections.emptyMap(), ByteArrays.EMPTY_ARRAY - ).expect( + ).expectAndThrow( new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), new IOException() ); @@ -432,22 +432,22 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final Map offsets = ImmutableMap.of(1, 3L); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout), HttpResponseStatus.ACCEPTED, Collections.emptyMap(), ByteArrays.EMPTY_ARRAY - ).expect( + ).expectAndRespond( new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.READING) - ).expect( + ).expectAndRespond( new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.PAUSED) - ).expect( + ).expectAndRespond( new RequestBuilder(HttpMethod.GET, "/offsets/current").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -460,22 +460,22 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_pauseAsync_threeIterations() throws Exception { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/pause").timeout(httpTimeout), HttpResponseStatus.ACCEPTED, Collections.emptyMap(), ByteArrays.EMPTY_ARRAY - ).expect( + ).expectAndRespond( new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.READING) - ).expect( + ).expectAndRespond( new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.READING) - ).expect( + ).expectAndRespond( new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -499,7 +499,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest { final Map retVal = ImmutableMap.of("foo", "xyz"); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/rowStats").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -512,7 +512,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_getMovingAveragesAsync_empty() throws Exception { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/rowStats").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -525,7 +525,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_getMovingAveragesAsync_null() throws Exception { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/rowStats").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -542,7 +542,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest new ParseExceptionReport("xyz", "foo", Collections.emptyList(), 123L) ); - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/unparseableEvents").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), @@ -555,7 +555,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest @Test public void test_getParseErrorsAsync_empty() throws Exception { - serviceClient.expect( + serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/unparseableEvents").timeout(httpTimeout), HttpResponseStatus.OK, Collections.emptyMap(), 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 f3df7997f53..2e2a0ee5fdc 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,8 +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.NoopOverlordClient; -import org.apache.druid.discovery.DruidLeaderClient; +import com.google.common.util.concurrent.Futures; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; @@ -43,6 +42,9 @@ import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.overlord.TestTaskRunner; import org.apache.druid.java.util.common.FileUtils; +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.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; @@ -52,6 +54,9 @@ import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; import org.apache.druid.server.security.AuthTestUtils; import org.easymock.EasyMock; +import org.jboss.netty.handler.codec.http.DefaultHttpResponse; +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; @@ -60,8 +65,11 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Map; /** @@ -79,6 +87,7 @@ public class WorkerTaskManagerTest private final boolean restoreTasksOnRestart; private WorkerTaskManager workerTaskManager; + private OverlordClient overlordClient; public WorkerTaskManagerTest(boolean restoreTasksOnRestart) { @@ -112,6 +121,7 @@ public class WorkerTaskManagerTest EasyMock.expect(taskActionClientFactory.create(EasyMock.anyObject())).andReturn(taskActionClient).anyTimes(); SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class); EasyMock.replay(taskActionClientFactory, taskActionClient, notifierFactory); + overlordClient = EasyMock.createMock(OverlordClient.class); return new WorkerTaskManager( jsonMapper, @@ -149,7 +159,7 @@ public class WorkerTaskManagerTest new NoopChatHandlerProvider(), testUtils.getRowIngestionMetersFactory(), new TestAppenderatorsManager(), - new NoopOverlordClient(), + overlordClient, null, null, null, @@ -160,7 +170,7 @@ public class WorkerTaskManagerTest location ), taskConfig, - EasyMock.createNiceMock(DruidLeaderClient.class) + overlordClient ) { @Override @@ -296,6 +306,7 @@ public class WorkerTaskManagerTest Map completeTasks; do { completeTasks = workerTaskManager.getCompletedTasks(); + Thread.sleep(10); } while (completeTasks.isEmpty()); Assert.assertEquals(1, completeTasks.size()); @@ -308,8 +319,155 @@ public class WorkerTaskManagerTest ); } + @Test(timeout = 30_000L) + public void test_completedTasksCleanup_running() throws Exception + { + final Task task = setUpCompletedTasksCleanupTest(); + + EasyMock.expect(overlordClient.taskStatuses(Collections.singleton(task.getId()))) + .andReturn(Futures.immediateFuture(ImmutableMap.of(task.getId(), TaskStatus.running(task.getId())))) + .once(); + EasyMock.replay(overlordClient); + + workerTaskManager.doCompletedTasksCleanup(); + Assert.assertEquals(1, workerTaskManager.getCompletedTasks().size()); + + EasyMock.verify(overlordClient); + } + + @Test(timeout = 30_000L) + public void test_completedTasksCleanup_noStatus() throws Exception + { + final Task task = setUpCompletedTasksCleanupTest(); + + EasyMock.expect(overlordClient.taskStatuses(Collections.singleton(task.getId()))) + .andReturn(Futures.immediateFuture(Collections.emptyMap())) + .once(); + EasyMock.replay(overlordClient); + + // Missing status (empty map) means we clean up the task. The idea is that this means the Overlord has *never* + // heard of it, so we should forget about it. + workerTaskManager.doCompletedTasksCleanup(); + Assert.assertEquals(0, workerTaskManager.getCompletedTasks().size()); + + EasyMock.verify(overlordClient); + } + + @Test(timeout = 30_000L) + public void test_completedTasksCleanup_success() throws Exception + { + final Task task = setUpCompletedTasksCleanupTest(); + + EasyMock.expect(overlordClient.taskStatuses(Collections.singleton(task.getId()))) + .andReturn(Futures.immediateFuture(ImmutableMap.of(task.getId(), TaskStatus.success(task.getId())))) + .once(); + EasyMock.replay(overlordClient); + + workerTaskManager.doCompletedTasksCleanup(); + Assert.assertEquals(0, workerTaskManager.getCompletedTasks().size()); + + EasyMock.verify(overlordClient); + } + + @Test(timeout = 30_000L) + public void test_completedTasksCleanup_404error() throws Exception + { + final Task task = setUpCompletedTasksCleanupTest(); + + EasyMock.expect(overlordClient.taskStatuses(Collections.singleton(task.getId()))) + .andReturn( + Futures.immediateFailedFuture( + new HttpResponseException( + new StringFullResponseHolder( + new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NOT_FOUND), + StandardCharsets.UTF_8 + ) + ) + ) + ) + .once(); + EasyMock.replay(overlordClient); + + // Ending size zero, because 404 means we assume the Overlord does not have the taskStatuses API. In this case + // we remove all completed task statuses periodically regardless of Overlord confirmation. + workerTaskManager.doCompletedTasksCleanup(); + Assert.assertEquals(0, workerTaskManager.getCompletedTasks().size()); + + EasyMock.verify(overlordClient); + } + + @Test(timeout = 30_000L) + public void test_completedTasksCleanup_500error() throws Exception + { + final Task task = setUpCompletedTasksCleanupTest(); + + EasyMock.expect(overlordClient.taskStatuses(Collections.singleton(task.getId()))) + .andReturn( + Futures.immediateFailedFuture( + new HttpResponseException( + new StringFullResponseHolder( + new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR), + StandardCharsets.UTF_8 + ) + ) + ) + ) + .once(); + EasyMock.replay(overlordClient); + + // HTTP 500 is ignored and no cleanup happens. + workerTaskManager.doCompletedTasksCleanup(); + Assert.assertEquals(1, workerTaskManager.getCompletedTasks().size()); + + EasyMock.verify(overlordClient); + } + + @Test(timeout = 30_000L) + public void test_completedTasksCleanup_ioException() throws Exception + { + final Task task = setUpCompletedTasksCleanupTest(); + + EasyMock.expect(overlordClient.taskStatuses(Collections.singleton(task.getId()))) + .andReturn(Futures.immediateFailedFuture(new IOException())) + .once(); + EasyMock.replay(overlordClient); + + // IOException is ignored and no cleanup happens. + workerTaskManager.doCompletedTasksCleanup(); + Assert.assertEquals(1, workerTaskManager.getCompletedTasks().size()); + + EasyMock.verify(overlordClient); + } + private NoopTask createNoopTask(String id) { return new NoopTask(id, null, null, 100, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, 0)); } + + /** + * Start the {@link #workerTaskManager}, submit a {@link NoopTask}, wait for it to be complete. Common preamble + * for various tests of {@link WorkerTaskManager#doCompletedTasksCleanup()}. + */ + private Task setUpCompletedTasksCleanupTest() throws Exception + { + final Task task = new NoopTask("id", null, null, 100, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, 0)); + + // Scheduled scheduleCompletedTasksCleanup will not run, because initialDelay is 1 minute, which is longer than + // the 30-second timeout of this test case. + workerTaskManager.start(); + workerTaskManager.assignTask(task); + + Map completeTasks; + do { + completeTasks = workerTaskManager.getCompletedTasks(); + Thread.sleep(10); + } while (completeTasks.isEmpty()); + + Assert.assertEquals(1, completeTasks.size()); + TaskAnnouncement announcement = completeTasks.get(task.getId()); + Assert.assertNotNull(announcement); + Assert.assertEquals(TaskState.SUCCESS, announcement.getStatus()); + + return task; + } } 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 e5868d8904d..5cedfb4312a 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 @@ -28,7 +28,6 @@ import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.TestingCluster; 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; import org.apache.druid.indexing.common.IndexingServiceCondition; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; @@ -48,6 +47,7 @@ import org.apache.druid.indexing.overlord.TestRemoteTaskRunnerConfig; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.StringUtils; +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; @@ -216,7 +216,7 @@ public class WorkerTaskMonitorTest taskConfig, cf, workerCuratorCoordinator, - EasyMock.createNiceMock(DruidLeaderClient.class) + EasyMock.createNiceMock(OverlordClient.class) ); } diff --git a/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java b/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java index cf81f78db61..75bbb32431c 100644 --- a/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java +++ b/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java @@ -20,7 +20,6 @@ package org.apache.druid.metadata; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.DateTimes; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -44,6 +43,11 @@ public interface TaskLookup COMPLETE } + /** + * Whether this lookup is guaranteed to not return any tasks. + */ + boolean isNil(); + TaskLookupType getType(); /** @@ -84,7 +88,7 @@ public interface TaskLookup return new CompleteTaskLookup(maxTaskStatuses, tasksCreatedPriorTo); } - private CompleteTaskLookup( + public CompleteTaskLookup( @Nullable Integer maxTaskStatuses, @Nullable DateTime tasksCreatedPriorTo ) @@ -98,12 +102,17 @@ public interface TaskLookup return tasksCreatedPriorTo != null; } - public CompleteTaskLookup withDurationBeforeNow(Duration durationBeforeNow) + /** + * If {@link #hasTaskCreatedTimeFilter()}, returns this instance. Otherwise, returns a copy with + * {@link #getTasksCreatedPriorTo()} based on the provided duration (before now). + */ + public CompleteTaskLookup withMinTimestampIfAbsent(DateTime minTimestamp) { - return CompleteTaskLookup.of( - maxTaskStatuses, - Preconditions.checkNotNull(durationBeforeNow, "durationBeforeNow") - ); + if (hasTaskCreatedTimeFilter()) { + return this; + } else { + return new CompleteTaskLookup(maxTaskStatuses, minTimestamp); + } } private static DateTime computeTimestampPriorToNow(Duration durationBeforeNow) @@ -131,6 +140,12 @@ public interface TaskLookup return TaskLookupType.COMPLETE; } + @Override + public boolean isNil() + { + return maxTaskStatuses != null && maxTaskStatuses == 0; + } + @Override public boolean equals(Object o) { @@ -171,6 +186,12 @@ public interface TaskLookup return TaskLookupType.ACTIVE; } + @Override + public boolean isNil() + { + return false; + } + @Override public int hashCode() { diff --git a/processing/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java b/processing/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java index 416c97e1840..52be34e78f9 100644 --- a/processing/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java @@ -33,6 +33,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.internal.matchers.ThrowableCauseMatcher; import org.junit.internal.matchers.ThrowableMessageMatcher; import java.util.ArrayList; @@ -206,6 +207,33 @@ public class FutureUtilsTest ); } + @Test + public void test_transform_error() + { + final ListenableFuture future = FutureUtils.transform( + Futures.immediateFuture("x"), + s -> { + throw new ISE("oops"); + } + ); + + Assert.assertTrue(future.isDone()); + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + future::get + ); + + MatcherAssert.assertThat( + e, + ThrowableCauseMatcher.hasCause(CoreMatchers.instanceOf(IllegalStateException.class)) + ); + + MatcherAssert.assertThat( + e, + ThrowableCauseMatcher.hasCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("oops"))) + ); + } + @Test public void test_transformAsync() throws Exception { diff --git a/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java b/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java index 2a2c5526926..39714b42de6 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java +++ b/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java @@ -56,6 +56,7 @@ public class TaskLookupTest Assert.assertNull(lookup.getMaxTaskStatuses()); Assert.assertFalse(lookup.hasTaskCreatedTimeFilter()); Assert.assertThrows(AssertionError.class, lookup::getTasksCreatedPriorTo); + Assert.assertFalse(lookup.isNil()); } @Test @@ -65,12 +66,29 @@ public class TaskLookupTest final DateTime timestampBeforeLookupCreated = DateTimes.nowUtc().minus(duration); final CompleteTaskLookup lookup = CompleteTaskLookup .of(null, null) - .withDurationBeforeNow(duration); + .withMinTimestampIfAbsent(timestampBeforeLookupCreated); Assert.assertNull(lookup.getMaxTaskStatuses()); Assert.assertTrue( timestampBeforeLookupCreated.isEqual(lookup.getTasksCreatedPriorTo()) || timestampBeforeLookupCreated.isBefore(lookup.getTasksCreatedPriorTo()) ); + Assert.assertFalse(lookup.isNil()); + } + + @Test + public void testWithDurationBeforeNow2() + { + final Duration duration = new Period("P1D").toStandardDuration(); + final DateTime timestampBeforeLookupCreated = DateTimes.nowUtc().minus(duration); + final CompleteTaskLookup lookup = + new CompleteTaskLookup(null, DateTimes.of("2000")) + .withMinTimestampIfAbsent(timestampBeforeLookupCreated); + Assert.assertNull(lookup.getMaxTaskStatuses()); + Assert.assertEquals( + DateTimes.of("2000"), + lookup.getTasksCreatedPriorTo() + ); + Assert.assertFalse(lookup.isNil()); } @Test @@ -86,6 +104,16 @@ public class TaskLookupTest timestampBeforeLookupCreated.isEqual(lookup.getTasksCreatedPriorTo()) || timestampBeforeLookupCreated.isBefore(lookup.getTasksCreatedPriorTo()) ); + Assert.assertFalse(lookup.isNil()); + } + + @Test + public void testZeroStatuses() + { + final CompleteTaskLookup lookup = CompleteTaskLookup.of(0, null); + Assert.assertNotNull(lookup.getMaxTaskStatuses()); + Assert.assertEquals(0, lookup.getMaxTaskStatuses().intValue()); + Assert.assertTrue(lookup.isNil()); } } @@ -98,6 +126,8 @@ public class TaskLookupTest final ActiveTaskLookup lookup2 = ActiveTaskLookup.getInstance(); Assert.assertEquals(lookup1, lookup2); Assert.assertSame(lookup1, lookup2); + Assert.assertFalse(lookup1.isNil()); + Assert.assertFalse(lookup2.isNil()); } @Test diff --git a/server/src/main/java/org/apache/druid/client/JsonParserIterator.java b/server/src/main/java/org/apache/druid/client/JsonParserIterator.java index e14e24c2231..237614fb4ff 100644 --- a/server/src/main/java/org/apache/druid/client/JsonParserIterator.java +++ b/server/src/main/java/org/apache/druid/client/JsonParserIterator.java @@ -27,6 +27,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryException; @@ -37,17 +38,15 @@ import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.utils.CloseableUtils; import javax.annotation.Nullable; -import java.io.Closeable; import java.io.IOException; import java.io.InputStream; -import java.util.Iterator; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -public class JsonParserIterator implements Iterator, Closeable +public class JsonParserIterator implements CloseableIterator { private static final Logger LOG = new Logger(JsonParserIterator.class); diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java index b69e0df2ed9..5873bd229db 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java @@ -35,7 +35,7 @@ import java.util.Objects; */ public class ClientCompactionTaskQuery implements ClientTaskQuery { - static final String TYPE = "compact"; + public static final String TYPE = "compact"; private final String id; private final String dataSource; diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java index eadda17afbe..1aeecfa4284 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java @@ -22,10 +22,11 @@ package org.apache.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.rpc.indexing.OverlordClient; /** * org.apache.druid.indexing.common.task.Task representations for clients. The magic conversion happens right - * at the moment of making a REST query: {@link HttpIndexingServiceClient#runTask} serializes ClientTaskQuery + * at the moment of making a REST query: {@link OverlordClient#runTask(String, Object)} serializes ClientTaskQuery * objects and org.apache.druid.indexing.overlord.http.OverlordResource.taskPost() deserializes * org.apache.druid.indexing.common.task.Task objects from the same bytes. Therefore JSON serialization fields of * ClientTaskQuery objects must match with those of the corresponding org.apache.druid.indexing.common.task.Task 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 deleted file mode 100644 index 821e4343901..00000000000 --- a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java +++ /dev/null @@ -1,472 +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 com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; -import com.google.common.collect.Iterables; -import com.google.inject.Inject; -import org.apache.druid.common.utils.IdUtils; -import org.apache.druid.discovery.DruidLeaderClient; -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.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularity; -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.StringFullResponseHolder; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.timeline.DataSegment; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import javax.ws.rs.core.MediaType; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class HttpIndexingServiceClient implements IndexingServiceClient -{ - private static final Logger log = new Logger(HttpIndexingServiceClient.class); - private final DruidLeaderClient druidLeaderClient; - private final ObjectMapper jsonMapper; - - @Inject - public HttpIndexingServiceClient( - ObjectMapper jsonMapper, - @IndexingService DruidLeaderClient druidLeaderClient - ) - { - this.jsonMapper = jsonMapper; - this.druidLeaderClient = druidLeaderClient; - } - - @Override - public void killUnusedSegments(String idPrefix, String dataSource, Interval interval) - { - final String taskId = IdUtils.newTaskId(idPrefix, ClientKillUnusedSegmentsTaskQuery.TYPE, dataSource, interval); - final ClientTaskQuery taskQuery = new ClientKillUnusedSegmentsTaskQuery(taskId, dataSource, interval, false); - runTask(taskId, taskQuery); - } - - @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 - ) - { - Preconditions.checkArgument(!segments.isEmpty(), "Expect non-empty segments to compact"); - - final String dataSource = segments.get(0).getDataSource(); - Preconditions.checkArgument( - segments.stream().allMatch(segment -> segment.getDataSource().equals(dataSource)), - "Segments must have the same dataSource" - ); - - context = context == null ? new HashMap<>() : context; - context.put("priority", compactionTaskPriority); - - final String taskId = IdUtils.newTaskId(idPrefix, ClientCompactionTaskQuery.TYPE, dataSource, null); - final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); - final ClientTaskQuery taskQuery = new ClientCompactionTaskQuery( - taskId, - dataSource, - new ClientCompactionIOConfig(ClientCompactionIntervalSpec.fromSegments(segments, segmentGranularity), dropExisting), - tuningConfig, - granularitySpec, - dimensionsSpec, - metricsSpec, - transformSpec, - context - ); - return runTask(taskId, taskQuery); - } - - @Override - public String runTask(String taskId, Object taskObject) - { - try { - // Warning, magic: here we may serialize ClientTaskQuery objects, but OverlordResource.taskPost() deserializes - // Task objects from the same data. See the comment for ClientTaskQuery for details. - final StringFullResponseHolder response = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/task") - .setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskObject)) - ); - - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - if (!Strings.isNullOrEmpty(response.getContent())) { - throw new ISE( - "Failed to post task[%s] with error[%s].", - taskId, - response.getContent() - ); - } else { - throw new ISE("Failed to post task[%s]. Please check overlord log", taskId); - } - } - - final Map resultMap = jsonMapper.readValue( - response.getContent(), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ); - final String returnedTaskId = (String) resultMap.get("task"); - Preconditions.checkState( - taskId.equals(returnedTaskId), - "Got a different taskId[%s]. Expected taskId[%s]", - returnedTaskId, - taskId - ); - return taskId; - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public String cancelTask(String taskId) - { - try { - final StringFullResponseHolder response = druidLeaderClient.go( - druidLeaderClient.makeRequest( - HttpMethod.POST, - StringUtils.format("/druid/indexer/v1/task/%s/shutdown", StringUtils.urlEncode(taskId)) - ) - ); - - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE("Failed to cancel task[%s]", taskId); - } - - final Map resultMap = jsonMapper.readValue( - response.getContent(), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ); - final String cancelledTaskId = (String) resultMap.get("task"); - Preconditions.checkNotNull(cancelledTaskId, "Null task id returned for task[%s]", taskId); - Preconditions.checkState( - taskId.equals(cancelledTaskId), - "Requested to cancel task[%s], but another task[%s] was cancelled!", - taskId, - cancelledTaskId - ); - return cancelledTaskId; - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public int getTotalWorkerCapacity() - { - try { - final StringFullResponseHolder response = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/workers") - .setHeader("Content-Type", MediaType.APPLICATION_JSON) - ); - - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE( - "Error while getting available cluster capacity. status[%s] content[%s]", - response.getStatus(), - response.getContent() - ); - - } - final Collection workers = jsonMapper.readValue( - response.getContent(), - new TypeReference>() {} - ); - - return workers.stream().mapToInt(workerInfo -> workerInfo.getWorker().getCapacity()).sum(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public int getTotalWorkerCapacityWithAutoScale() - { - try { - final StringFullResponseHolder response = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/totalWorkerCapacity") - .setHeader("Content-Type", MediaType.APPLICATION_JSON) - ); - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE( - "Error while getting total worker capacity. status[%s] content[%s]", - response.getStatus(), - response.getContent() - ); - } - final IndexingTotalWorkerCapacityInfo indexingTotalWorkerCapacityInfo = jsonMapper.readValue( - response.getContent(), - new TypeReference() {} - ); - return indexingTotalWorkerCapacityInfo.getMaximumCapacityWithAutoScale(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public List getActiveTasks() - { - // Must retrieve waiting, then pending, then running, so if tasks move from one state to the next between - // calls then we still catch them. (Tasks always go waiting -> pending -> running.) - // - // Consider switching to new-style /druid/indexer/v1/tasks API in the future. - final List tasks = new ArrayList<>(); - final Set taskIdsSeen = new HashSet<>(); - - final Iterable activeTasks = Iterables.concat( - getTasks("waitingTasks"), - getTasks("pendingTasks"), - getTasks("runningTasks") - ); - - for (TaskStatusPlus task : activeTasks) { - // Use taskIdsSeen to prevent returning the same task ID more than once (if it hops from 'pending' to 'running', - // for example, and we see it twice.) - if (taskIdsSeen.add(task.getId())) { - tasks.add(task); - } - } - - return tasks; - } - - private List getTasks(String endpointSuffix) - { - try { - final StringFullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/%s", endpointSuffix)) - ); - - if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE("Error while fetching the status of tasks"); - } - - return jsonMapper.readValue( - responseHolder.getContent(), - new TypeReference>() - { - } - ); - } - catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - } - - @Override - public TaskStatusResponse getTaskStatus(String taskId) - { - try { - final StringFullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format( - "/druid/indexer/v1/task/%s/status", - StringUtils.urlEncode(taskId) - )) - ); - - return jsonMapper.readValue( - responseHolder.getContent(), - new TypeReference() - { - } - ); - } - catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - } - - @Override - @Nullable - public TaskStatusPlus getLastCompleteTask() - { - final List completeTaskStatuses = getTasks("completeTasks?n=1"); - return completeTaskStatuses.isEmpty() ? null : completeTaskStatuses.get(0); - } - - @Override - public TaskPayloadResponse getTaskPayload(String taskId) - { - try { - final StringFullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest( - HttpMethod.GET, - StringUtils.format("/druid/indexer/v1/task/%s", StringUtils.urlEncode(taskId)) - ) - ); - - return jsonMapper.readValue( - responseHolder.getContent(), - new TypeReference() - { - } - ); - } - catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - } - - @Nullable - @Override - public Map getTaskReport(String taskId) - { - try { - final StringFullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest( - HttpMethod.GET, - StringUtils.format("/druid/indexer/v1/task/%s/reports", StringUtils.urlEncode(taskId)) - ) - ); - - if (responseHolder.getContent().length() == 0 || !HttpResponseStatus.OK.equals(responseHolder.getStatus())) { - if (responseHolder.getStatus() == HttpResponseStatus.NOT_FOUND) { - log.info("Report not found for taskId [%s] because [%s]", taskId, responseHolder.getContent()); - } else { - // also log other non-ok statuses: - log.info( - "Non OK response status [%s] for taskId [%s] because [%s]", - responseHolder.getStatus(), - taskId, - responseHolder.getContent() - ); - } - return null; - } - - return jsonMapper.readValue( - responseHolder.getContent(), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ); - } - catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - } - - @Override - public Map> getLockedIntervals(Map minTaskPriority) - { - try { - final StringFullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/lockedIntervals") - .setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(minTaskPriority)) - ); - - final Map> response = jsonMapper.readValue( - responseHolder.getContent(), - new TypeReference>>() - { - } - ); - return response == null ? Collections.emptyMap() : response; - } - catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - } - - @Override - public SamplerResponse sample(SamplerSpec samplerSpec) - { - try { - final StringFullResponseHolder response = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/sampler") - .setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(samplerSpec)) - ); - - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - if (!Strings.isNullOrEmpty(response.getContent())) { - throw new ISE( - "Failed to sample with sampler spec[%s], response[%s].", - samplerSpec, - response.getContent() - ); - } else { - throw new ISE("Failed to sample with sampler spec[%s]. Please check overlord log", samplerSpec); - } - } - - return jsonMapper.readValue(response.getContent(), SamplerResponse.class); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public int killPendingSegments(String dataSource, DateTime end) - { - final String endPoint = StringUtils.format( - "/druid/indexer/v1/pendingSegments/%s?interval=%s", - StringUtils.urlEncode(dataSource), - new Interval(DateTimes.MIN, end) - ); - try { - final StringFullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.DELETE, endPoint) - ); - - if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE("Error while killing pendingSegments of dataSource[%s] created until [%s]", dataSource, end); - } - - final Map resultMap = jsonMapper.readValue( - responseHolder.getContent(), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ); - - final Object numDeletedObject = resultMap.get("numDeleted"); - return (Integer) Preconditions.checkNotNull(numDeletedObject, "numDeletedObject"); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } -} 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 deleted file mode 100644 index 2658d57d237..00000000000 --- a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java +++ /dev/null @@ -1,102 +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.rpc.indexing.OverlordClient; -import org.apache.druid.timeline.DataSegment; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.List; -import java.util.Map; - -/** - * High-level IndexingServiceClient client. - * - * New use cases should prefer {@link OverlordClient}. - */ -public interface IndexingServiceClient -{ - void killUnusedSegments(String idPrefix, String dataSource, Interval interval); - - int killPendingSegments(String dataSource, DateTime end); - - 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 - ); - - /** - * Gets the total worker capacity of the current state of the cluster. This can be -1 if it cannot be determined. - */ - int getTotalWorkerCapacity(); - - /** - * Gets the total worker capacity of the cluster including auto scaling capability (scaling to max workers). - * This can be -1 if it cannot be determined or if auto scaling is not configured. - */ - int getTotalWorkerCapacityWithAutoScale(); - - String runTask(String taskId, Object taskObject); - - String cancelTask(String taskId); - - /** - * Gets all tasks that are waiting, pending, or running. - */ - List getActiveTasks(); - - TaskStatusResponse getTaskStatus(String taskId); - - @Nullable - TaskStatusPlus getLastCompleteTask(); - - @Nullable - TaskPayloadResponse getTaskPayload(String taskId); - - @Nullable - Map getTaskReport(String taskId); - - /** - * Gets a List of Intervals locked by higher priority tasks for each datasource. - * - * @param minTaskPriority Minimum task priority for each datasource. Only the - * 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. - */ - Map> getLockedIntervals(Map minTaskPriority); - - SamplerResponse sample(SamplerSpec samplerSpec); -} diff --git a/server/src/main/java/org/apache/druid/client/indexing/IndexingTotalWorkerCapacityInfo.java b/server/src/main/java/org/apache/druid/client/indexing/IndexingTotalWorkerCapacityInfo.java index bd3194d41ab..fb89a64202c 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/IndexingTotalWorkerCapacityInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/IndexingTotalWorkerCapacityInfo.java @@ -22,6 +22,8 @@ package org.apache.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Objects; + /** * Should be synchronized with org.apache.druid.indexing.overlord.http.TotalWorkerCapacityResponse */ @@ -59,4 +61,33 @@ public class IndexingTotalWorkerCapacityInfo { return maximumCapacityWithAutoScale; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IndexingTotalWorkerCapacityInfo that = (IndexingTotalWorkerCapacityInfo) o; + return currentClusterCapacity == that.currentClusterCapacity + && maximumCapacityWithAutoScale == that.maximumCapacityWithAutoScale; + } + + @Override + public int hashCode() + { + return Objects.hash(currentClusterCapacity, maximumCapacityWithAutoScale); + } + + @Override + public String toString() + { + return "IndexingTotalWorkerCapacityInfo{" + + "currentClusterCapacity=" + currentClusterCapacity + + ", maximumCapacityWithAutoScale=" + maximumCapacityWithAutoScale + + '}'; + } } diff --git a/server/src/main/java/org/apache/druid/client/indexing/IndexingWorker.java b/server/src/main/java/org/apache/druid/client/indexing/IndexingWorker.java index 129117533c2..04251b41dc5 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/IndexingWorker.java +++ b/server/src/main/java/org/apache/druid/client/indexing/IndexingWorker.java @@ -22,6 +22,8 @@ package org.apache.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Objects; + /** * Should be synchronized with org.apache.druid.indexing.worker.Worker */ @@ -78,4 +80,38 @@ public class IndexingWorker { return version; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IndexingWorker that = (IndexingWorker) o; + return capacity == that.capacity && Objects.equals(scheme, that.scheme) && Objects.equals( + host, + that.host + ) && Objects.equals(ip, that.ip) && Objects.equals(version, that.version); + } + + @Override + public int hashCode() + { + return Objects.hash(scheme, host, ip, capacity, version); + } + + @Override + public String toString() + { + return "IndexingWorker{" + + "scheme='" + scheme + '\'' + + ", host='" + host + '\'' + + ", ip='" + ip + '\'' + + ", capacity=" + capacity + + ", version='" + version + '\'' + + '}'; + } } diff --git a/server/src/main/java/org/apache/druid/client/indexing/IndexingWorkerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/IndexingWorkerInfo.java index 8aa43439dd1..87de8c49a9d 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/IndexingWorkerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/IndexingWorkerInfo.java @@ -25,6 +25,7 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Collection; +import java.util.Objects; import java.util.Set; /** @@ -75,11 +76,6 @@ public class IndexingWorkerInfo return availabilityGroups; } - public int getAvailableCapacity() - { - return getWorker().getCapacity() - getCurrCapacityUsed(); - } - @JsonProperty("runningTasks") public Collection getRunningTasks() { @@ -97,4 +93,48 @@ public class IndexingWorkerInfo { return blacklistedUntil; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IndexingWorkerInfo that = (IndexingWorkerInfo) o; + return currCapacityUsed == that.currCapacityUsed + && Objects.equals(worker, that.worker) + && Objects.equals(availabilityGroups, that.availabilityGroups) + && Objects.equals(runningTasks, that.runningTasks) + && Objects.equals(lastCompletedTaskTime, that.lastCompletedTaskTime) + && Objects.equals(blacklistedUntil, that.blacklistedUntil); + } + + @Override + public int hashCode() + { + return Objects.hash( + worker, + currCapacityUsed, + availabilityGroups, + runningTasks, + lastCompletedTaskTime, + blacklistedUntil + ); + } + + @Override + public String toString() + { + return "IndexingWorkerInfo{" + + "worker=" + worker + + ", currCapacityUsed=" + currCapacityUsed + + ", availabilityGroups=" + availabilityGroups + + ", runningTasks=" + runningTasks + + ", lastCompletedTaskTime=" + lastCompletedTaskTime + + ", blacklistedUntil=" + blacklistedUntil + + '}'; + } } diff --git a/server/src/main/java/org/apache/druid/guice/IndexingServiceDiscoveryModule.java b/server/src/main/java/org/apache/druid/guice/IndexingServiceDiscoveryModule.java deleted file mode 100644 index 3e480781391..00000000000 --- a/server/src/main/java/org/apache/druid/guice/IndexingServiceDiscoveryModule.java +++ /dev/null @@ -1,58 +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.guice; - -import com.google.inject.Binder; -import com.google.inject.Module; -import com.google.inject.Provides; -import org.apache.druid.client.indexing.IndexingService; -import org.apache.druid.client.indexing.IndexingServiceSelectorConfig; -import org.apache.druid.discovery.DruidLeaderClient; -import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeRole; -import org.apache.druid.guice.annotations.EscalatedGlobal; -import org.apache.druid.java.util.http.client.HttpClient; - -/** - */ -public class IndexingServiceDiscoveryModule implements Module -{ - @Override - public void configure(Binder binder) - { - JsonConfigProvider.bind(binder, "druid.selectors.indexing", IndexingServiceSelectorConfig.class); - } - - @Provides - @IndexingService - @ManageLifecycle - public DruidLeaderClient getLeaderHttpClient( - @EscalatedGlobal HttpClient httpClient, - DruidNodeDiscoveryProvider druidNodeDiscoveryProvider - ) - { - return new DruidLeaderClient( - httpClient, - druidNodeDiscoveryProvider, - NodeRole.OVERLORD, - "/druid/indexer/v1/leader" - ); - } -} diff --git a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java index d0f0b453f28..de4f3abbc8d 100644 --- a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java +++ b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java @@ -31,7 +31,6 @@ import org.apache.druid.guice.DruidSecondaryModule; import org.apache.druid.guice.ExpressionModule; import org.apache.druid.guice.ExtensionsModule; import org.apache.druid.guice.FirehoseModule; -import org.apache.druid.guice.IndexingServiceDiscoveryModule; import org.apache.druid.guice.JacksonConfigManagerModule; import org.apache.druid.guice.JavaScriptModule; import org.apache.druid.guice.LifecycleModule; @@ -123,7 +122,6 @@ public class CoreInjectorBuilder extends DruidInjectorBuilder new MetadataConfigModule(), new DerbyMetadataStorageDruidModule(), new JacksonConfigManagerModule(), - new IndexingServiceDiscoveryModule(), new CoordinatorDiscoveryModule(), new LocalDataStorageDruidModule(), new TombstoneDataStorageModule(), 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 index dac5dfc2263..0cb77ee7045 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -20,20 +20,30 @@ package org.apache.druid.rpc.indexing; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery; +import org.apache.druid.client.indexing.ClientTaskQuery; +import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; +import org.apache.druid.client.indexing.IndexingWorkerInfo; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.rpc.ServiceRetryPolicy; +import org.joda.time.Interval; +import javax.annotation.Nullable; +import java.net.URI; +import java.util.List; 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}. * @@ -43,17 +53,141 @@ import java.util.Set; */ public interface OverlordClient { + /** + * Contact the Overlord that we believe to be the leader, and return the result of its + * {@code /druid/indexer/v1/leader} API. This may be a different Overlord server than the one we contacted, if + * a leader change happened since the last time we updated our sense of who the leader is. + */ + ListenableFuture findCurrentLeader(); + + /** + * Run a task with the provided ID and payload. The payload must be convertible by an + * {@link com.fasterxml.jackson.databind.ObjectMapper} into a Task object. This method does not take Task objects + * directly, because Task is in the indexing-service package. + * + * @param taskId task ID + * @param taskObject task payload + */ ListenableFuture runTask(String taskId, Object taskObject); + /** + * Run a "kill" task for a particular datasource and interval. Shortcut to {@link #runTask(String, Object)}. + * + * The kill task deletes all unused segment records from deep storage and the metadata store. The task runs + * asynchronously after the API call returns. The resolved future is the ID of the task, which can be used to + * monitor its progress through the {@link #taskStatus(String)} API. + * + * @param idPrefix Descriptive prefix to include at the start of task IDs + * @param dataSource Datasource to kill + * @param interval Interval to kill + * + * @return future with task ID + */ + default ListenableFuture runKillTask(String idPrefix, String dataSource, Interval interval) + { + final String taskId = IdUtils.newTaskId(idPrefix, ClientKillUnusedSegmentsTaskQuery.TYPE, dataSource, interval); + final ClientTaskQuery taskQuery = new ClientKillUnusedSegmentsTaskQuery(taskId, dataSource, interval, false); + return FutureUtils.transform(runTask(taskId, taskQuery), ignored -> taskId); + } + + /** + * Cancel a task. + * + * @param taskId task ID + */ ListenableFuture cancelTask(String taskId); + /** + * Return {@link TaskStatusPlus} for all tasks matching a set of optional search parameters. + * + * Complete tasks are returned in descending order by creation timestamp. Active tasks are returned in no + * particular order. + * + * @param state task state: may be "pending", "waiting", "running", or "complete" + * @param dataSource datasource + * @param maxCompletedTasks maximum number of completed tasks to return. If zero, no complete tasks are returned. + * If null, all complete tasks within {@code druid.indexer.storage.recentlyFinishedThreshold} + * are returned. This parameter does not affect the number of active tasks returned. + * + * @return list of tasks that match the search parameters + */ + ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ); + + /** + * Return {@link TaskStatus} for a set of task IDs. + * + * @param taskIds task IDs + * + * @return map of task ID to status for known tasks. Unknown tasks are not included in the returned map. + */ ListenableFuture> taskStatuses(Set taskIds); + /** + * Returns {@link TaskStatusResponse} for a particular task ID. This includes somewhat more information than + * the {@link TaskStatus} returned by {@link #taskStatuses(Set)}. + */ ListenableFuture taskStatus(String taskId); + /** + * Returns the report object for a task as a map. Certain task types offer live reports; for these task types, + * this method may return a task report while the task is running. Certain task types only write reports upon + * successful completion. Certain other task types do not write reports at all. + * + * Returns a {@link org.apache.druid.rpc.HttpResponseException} with code + * {@link javax.ws.rs.core.Response.Status#NOT_FOUND} if there is no report available for some reason. + */ ListenableFuture> taskReportAsMap(String taskId); + /** + * Returns the payload for a task as an instance of {@link ClientTaskQuery}. This method only works for tasks + * that have a {@link ClientTaskQuery} model or are subclasses of {@link ClientTaskQuery}. + */ ListenableFuture taskPayload(String taskId); + /** + * Returns all current supervisor statuses. + */ + ListenableFuture> supervisorStatuses(); + + /** + * Returns a list of intervals locked by higher priority tasks for each datasource. + * + * @param minTaskPriority Minimum task priority for each datasource. Only the intervals that are locked by tasks with + * equal or higher priority than this are returned. + * + * @return Map from dtasource name to list of intervals locked by tasks that have priority greater than or equal to + * the {@code minTaskPriority} for that datasource. + */ + ListenableFuture>> findLockedIntervals(Map minTaskPriority); + + /** + * Deletes pending segment records from the metadata store for a particular datasource. Records with + * {@code created_date} within the provided {@code interval} are deleted; other records are left alone. + * Deletion is done synchronously with the API call. When the future resolves, the deletion is complete. + * + * @param dataSource datasource name + * @param interval created time interval + * + * @return number of pending segments deleted + */ + ListenableFuture killPendingSegments(String dataSource, Interval interval); + + /** + * Returns information about workers. + */ + ListenableFuture> getWorkers(); + + /** + * Returns total worker capacity details. + */ + ListenableFuture getTotalWorkerCapacity(); + + /** + * Returns a copy of this client with a different retry policy. + */ 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 index 5c28d87a8d1..dcfe0b2c589 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -22,22 +22,37 @@ 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.Futures; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.JsonParserIterator; +import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; +import org.apache.druid.client.indexing.IndexingWorkerInfo; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; 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.InputStreamResponseHandler; +import org.apache.druid.java.util.http.client.response.StringFullResponseHandler; 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 org.joda.time.Interval; -import java.io.IOException; +import javax.annotation.Nullable; +import java.io.InputStream; +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Set; @@ -55,6 +70,25 @@ public class OverlordClientImpl implements OverlordClient this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); } + @Override + public ListenableFuture findCurrentLeader() + { + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/leader"), + new StringFullResponseHandler(StandardCharsets.UTF_8) + ), + holder -> { + try { + return new URI(holder.getContent()); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + ); + } + @Override public ListenableFuture runTask(final String taskId, final Object taskObject) { @@ -65,7 +99,8 @@ public class OverlordClientImpl implements OverlordClient new BytesFullResponseHandler() ), holder -> { - final Map map = deserialize(holder, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); + final Map map = + JacksonUtils.readValue(jsonMapper, holder.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); final String returnedTaskId = (String) map.get("task"); Preconditions.checkState( @@ -91,6 +126,39 @@ public class OverlordClientImpl implements OverlordClient ); } + @Override + public ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ) + { + final StringBuilder pathBuilder = new StringBuilder("/druid/indexer/v1/tasks"); + int params = 0; + + if (state != null) { + pathBuilder.append('?').append("state=").append(StringUtils.urlEncode(state)); + params++; + } + + if (dataSource != null) { + pathBuilder.append(params == 0 ? '?' : '&').append("datasource=").append(StringUtils.urlEncode(dataSource)); + params++; + } + + if (maxCompletedTasks != null) { + pathBuilder.append(params == 0 ? '?' : '&').append("maxCompletedTasks=").append(maxCompletedTasks); + } + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, pathBuilder.toString()), + new InputStreamResponseHandler() + ), + in -> asJsonParserIterator(in, TaskStatusPlus.class) + ); + } + @Override public ListenableFuture> taskStatuses(final Set taskIds) { @@ -100,7 +168,8 @@ public class OverlordClientImpl implements OverlordClient .jsonContent(jsonMapper, taskIds), new BytesFullResponseHandler() ), - holder -> deserialize(holder, new TypeReference>() {}) + holder -> + JacksonUtils.readValue(jsonMapper, holder.getContent(), new TypeReference>() {}) ); } @@ -114,7 +183,30 @@ public class OverlordClientImpl implements OverlordClient new RequestBuilder(HttpMethod.GET, path), new BytesFullResponseHandler() ), - holder -> deserialize(holder, TaskStatusResponse.class) + holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), TaskStatusResponse.class) + ); + } + + @Override + public ListenableFuture>> findLockedIntervals(Map minTaskPriority) + { + final String path = "/druid/indexer/v1/lockedIntervals"; + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.POST, path) + .jsonContent(jsonMapper, minTaskPriority), + new BytesFullResponseHandler() + ), + holder -> { + final Map> response = JacksonUtils.readValue( + jsonMapper, + holder.getContent(), + new TypeReference>>() {} + ); + + return response == null ? Collections.emptyMap() : response; + } ); } @@ -128,12 +220,82 @@ public class OverlordClientImpl implements OverlordClient new RequestBuilder(HttpMethod.GET, path), new BytesFullResponseHandler() ), - holder -> deserialize(holder, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT) + holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT) ); } @Override - public ListenableFuture taskPayload(String taskId) + public ListenableFuture> supervisorStatuses() + { + final String path = "/druid/indexer/v1/supervisor?system"; + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new InputStreamResponseHandler() + ), + in -> asJsonParserIterator(in, SupervisorStatus.class) + ); + } + + @Override + public ListenableFuture> getWorkers() + { + final String path = "/druid/indexer/v1/workers"; + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> + JacksonUtils.readValue(jsonMapper, holder.getContent(), new TypeReference>() {}) + ); + } + + @Override + public ListenableFuture getTotalWorkerCapacity() + { + final String path = "/druid/indexer/v1/totalWorkerCapacity"; + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), IndexingTotalWorkerCapacityInfo.class) + ); + } + + @Override + public ListenableFuture killPendingSegments(String dataSource, Interval interval) + { + final String path = StringUtils.format( + "/druid/indexer/v1/pendingSegments/%s?interval=%s", + StringUtils.urlEncode(dataSource), + StringUtils.urlEncode(interval.toString()) + ); + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.DELETE, path), + new BytesFullResponseHandler() + ), + holder -> { + final Map resultMap = JacksonUtils.readValue( + jsonMapper, + holder.getContent(), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + + final Object numDeletedObject = resultMap.get("numDeleted"); + return (Integer) Preconditions.checkNotNull(numDeletedObject, "numDeletedObject"); + } + ); + } + + @Override + public ListenableFuture taskPayload(final String taskId) { final String path = StringUtils.format("/druid/indexer/v1/task/%s", StringUtils.urlEncode(taskId)); @@ -142,9 +304,7 @@ public class OverlordClientImpl implements OverlordClient new RequestBuilder(HttpMethod.GET, path), new BytesFullResponseHandler() ), - holder -> deserialize(holder, new TypeReference() - { - }) + holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), TaskPayloadResponse.class) ); } @@ -154,23 +314,15 @@ public class OverlordClientImpl implements OverlordClient return new OverlordClientImpl(client.withRetryPolicy(retryPolicy), jsonMapper); } - private T deserialize(final BytesFullResponseHolder bytesHolder, final Class clazz) + private JsonParserIterator asJsonParserIterator(final InputStream in, 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); - } + return new JsonParserIterator<>( + jsonMapper.getTypeFactory().constructType(clazz), + Futures.immediateFuture(in), + "", // We don't know URL at this point, but it's OK to use empty; it's used for logs/errors + null, + "", // We don't know host at this point, but it's OK to use empty; it's used for logs/errors + jsonMapper + ); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index c9d334968ef..dbc04b06bd5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -38,7 +38,6 @@ import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.ServerInventoryView; import org.apache.druid.client.coordinator.Coordinator; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.curator.discovery.ServiceAnnouncer; import org.apache.druid.discovery.DruidLeaderSelector; @@ -58,6 +57,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.balancer.BalancerStrategy; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; @@ -140,7 +140,7 @@ public class DruidCoordinator private final MetadataRuleManager metadataRuleManager; private final ServiceEmitter emitter; - private final IndexingServiceClient indexingServiceClient; + private final OverlordClient overlordClient; private final ScheduledExecutorService exec; private final LoadQueueTaskMaster taskMaster; private final ConcurrentHashMap loadManagementPeons = new ConcurrentHashMap<>(); @@ -187,7 +187,7 @@ public class DruidCoordinator MetadataRuleManager metadataRuleManager, ServiceEmitter emitter, ScheduledExecutorFactory scheduledExecutorFactory, - IndexingServiceClient indexingServiceClient, + OverlordClient overlordClient, LoadQueueTaskMaster taskMaster, SegmentLoadQueueManager loadQueueManager, ServiceAnnouncer serviceAnnouncer, @@ -208,7 +208,7 @@ public class DruidCoordinator this.serverInventoryView = serverInventoryView; this.metadataRuleManager = metadataRuleManager; this.emitter = emitter; - this.indexingServiceClient = indexingServiceClient; + this.overlordClient = overlordClient; this.taskMaster = taskMaster; this.serviceAnnouncer = serviceAnnouncer; this.self = self; @@ -461,7 +461,7 @@ public class DruidCoordinator config.getCoordinatorPeriod() ) ); - if (indexingServiceClient != null) { + if (overlordClient != null) { dutiesRunnables.add( new DutiesRunnable( makeIndexingServiceDuties(), @@ -619,7 +619,7 @@ public class DruidCoordinator { List compactSegmentsDutyFromCustomGroups = getCompactSegmentsDutyFromCustomGroups(); if (compactSegmentsDutyFromCustomGroups.isEmpty()) { - return new CompactSegments(config, compactionSegmentSearchPolicy, indexingServiceClient); + return new CompactSegments(config, compactionSegmentSearchPolicy, overlordClient); } else { if (compactSegmentsDutyFromCustomGroups.size() > 1) { log.warn( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/KillStalePendingSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/KillStalePendingSegments.java index f9da22e9a71..c2cd2c1fe68 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/KillStalePendingSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/KillStalePendingSegments.java @@ -20,14 +20,17 @@ package org.apache.druid.server.coordinator; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.inject.Inject; -import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.duty.CoordinatorDuty; import org.joda.time.DateTime; +import org.joda.time.Interval; import org.joda.time.Period; import java.util.ArrayList; @@ -38,28 +41,37 @@ public class KillStalePendingSegments implements CoordinatorDuty private static final Logger log = new Logger(KillStalePendingSegments.class); private static final Period KEEP_PENDING_SEGMENTS_OFFSET = new Period("P1D"); - private final IndexingServiceClient indexingServiceClient; + private final OverlordClient overlordClient; @Inject - public KillStalePendingSegments(IndexingServiceClient indexingServiceClient) + public KillStalePendingSegments(OverlordClient overlordClient) { - this.indexingServiceClient = indexingServiceClient; + this.overlordClient = overlordClient; } @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { final List createdTimes = new ArrayList<>(); + + // Include one complete status so we can get the time of the last-created complete task. (The Overlord API returns + // complete tasks in descending order of created_date.) + final List statuses = + ImmutableList.copyOf(FutureUtils.getUnchecked(overlordClient.taskStatuses(null, null, 1), true)); createdTimes.add( - indexingServiceClient - .getActiveTasks() + statuses .stream() + .filter(status -> status.getStatusCode() == null || !status.getStatusCode().isComplete()) .map(TaskStatusPlus::getCreatedTime) .min(Comparators.naturalNullsFirst()) .orElse(DateTimes.nowUtc()) // If there are no active tasks, this returns the current time. ); - final TaskStatusPlus completeTaskStatus = indexingServiceClient.getLastCompleteTask(); + final TaskStatusPlus completeTaskStatus = + statuses.stream() + .filter(status -> status != null && status.getStatusCode().isComplete()) + .findFirst() + .orElse(null); if (completeTaskStatus != null) { createdTimes.add(completeTaskStatus.getCreatedTime()); } @@ -74,9 +86,16 @@ public class KillStalePendingSegments implements CoordinatorDuty final DateTime stalePendingSegmentsCutoffCreationTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); for (String dataSource : params.getUsedSegmentsTimelinesPerDataSource().keySet()) { if (!params.getCoordinatorDynamicConfig().getDataSourcesToNotKillStalePendingSegmentsIn().contains(dataSource)) { + final int pendingSegmentsKilled = FutureUtils.getUnchecked( + overlordClient.killPendingSegments( + dataSource, + new Interval(DateTimes.MIN, stalePendingSegmentsCutoffCreationTime) + ), + true + ); log.info( "Killed [%d] pendingSegments created until [%s] for dataSource[%s]", - indexingServiceClient.killPendingSegments(dataSource, stalePendingSegmentsCutoffCreationTime), + pendingSegmentsKilled, stalePendingSegmentsCutoffCreationTime, dataSource ); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 0381a70ba24..4dd7cb5dd75 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -22,20 +22,31 @@ package org.apache.druid.server.coordinator.duty; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.inject.Inject; +import org.apache.druid.client.indexing.ClientCompactionIOConfig; +import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; import org.apache.druid.client.indexing.ClientCompactionTaskDimensionsSpec; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; 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.IndexingServiceClient; +import org.apache.druid.client.indexing.ClientTaskQuery; +import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.rpc.HttpResponseException; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CompactionStatistics; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; @@ -48,30 +59,37 @@ import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; public class CompactSegments implements CoordinatorCustomDuty { - /** Must be the same as org.apache.druid.indexing.common.task.CompactionTask.TYPE. */ + /** + * Must be the same as org.apache.druid.indexing.common.task.CompactionTask.TYPE. + */ public static final String COMPACTION_TASK_TYPE = "compact"; - /** Must be the same as org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY */ + /** + * Must be the same as org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY + */ public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState"; private static final Logger LOG = new Logger(CompactSegments.class); private final CompactionSegmentSearchPolicy policy; private final boolean skipLockedIntervals; - private final IndexingServiceClient indexingServiceClient; + private final OverlordClient overlordClient; // This variable is updated by the Coordinator thread executing duties and // read by HTTP threads processing Coordinator API calls. @@ -82,11 +100,11 @@ public class CompactSegments implements CoordinatorCustomDuty public CompactSegments( @JacksonInject DruidCoordinatorConfig config, @JacksonInject CompactionSegmentSearchPolicy policy, - @JacksonInject IndexingServiceClient indexingServiceClient + @JacksonInject OverlordClient overlordClient ) { this.policy = policy; - this.indexingServiceClient = indexingServiceClient; + this.overlordClient = overlordClient; this.skipLockedIntervals = config.getCompactionSkipLockedIntervals(); resetCompactionSnapshot(); @@ -99,6 +117,12 @@ public class CompactSegments implements CoordinatorCustomDuty return skipLockedIntervals; } + @VisibleForTesting + public OverlordClient getOverlordClient() + { + return overlordClient; + } + @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { @@ -128,9 +152,12 @@ public class CompactSegments implements CoordinatorCustomDuty // Fetch currently running compaction tasks int busyCompactionTaskSlots = 0; - final List compactionTasks = filterNonCompactionTasks(indexingServiceClient.getActiveTasks()); + final CloseableIterator activeTasks = + FutureUtils.getUnchecked(overlordClient.taskStatuses(null, null, 0), true); + final List compactionTasks = filterNonCompactionTasks(activeTasks); for (TaskStatusPlus status : compactionTasks) { - final TaskPayloadResponse response = indexingServiceClient.getTaskPayload(status.getId()); + final TaskPayloadResponse response = + FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true); if (response == null) { throw new ISE("Could not find payload for active compaction task[%s]", status.getId()); } else if (!COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) { @@ -225,7 +252,7 @@ public class CompactSegments implements CoordinatorCustomDuty "Cancelling task [%s] as task segmentGranularity is [%s] but compaction config segmentGranularity is [%s]", compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity ); - indexingServiceClient.cancelTask(compactionTaskQuery.getId()); + overlordClient.cancelTask(compactionTaskQuery.getId()); return true; } @@ -261,7 +288,7 @@ public class CompactSegments implements CoordinatorCustomDuty ) ); final Map> datasourceToLockedIntervals = - new HashMap<>(indexingServiceClient.getLockedIntervals(minTaskPriority)); + new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(minTaskPriority), true)); LOG.debug( "Skipping the following intervals for Compaction as they are currently locked: %s", datasourceToLockedIntervals @@ -278,7 +305,8 @@ public class CompactSegments implements CoordinatorCustomDuty static int findMaxNumTaskSlotsUsedByOneCompactionTask(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) { if (isParallelMode(tuningConfig)) { - @Nullable Integer maxNumConcurrentSubTasks = tuningConfig.getMaxNumConcurrentSubTasks(); + @Nullable + Integer maxNumConcurrentSubTasks = tuningConfig.getMaxNumConcurrentSubTasks(); // Max number of task slots used in parallel mode = maxNumConcurrentSubTasks + 1 (supervisor task) return (maxNumConcurrentSubTasks == null ? 1 : maxNumConcurrentSubTasks) + 1; } else { @@ -308,32 +336,61 @@ public class CompactSegments implements CoordinatorCustomDuty return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec; } - private static List filterNonCompactionTasks(List taskStatuses) + private static List filterNonCompactionTasks(CloseableIterator taskStatuses) { - return taskStatuses - .stream() - .filter(status -> { - final String taskType = status.getType(); - // taskType can be null if middleManagers are running with an older version. Here, we consevatively regard - // the tasks of the unknown taskType as the compactionTask. This is because it's important to not run - // compactionTasks more than the configured limit at any time which might impact to the ingestion - // performance. - return taskType == null || COMPACTION_TASK_TYPE.equals(taskType); - }) - .collect(Collectors.toList()); + final List retVal = new ArrayList<>(); + + try (final Closer closer = Closer.create()) { + closer.register(taskStatuses); + while (taskStatuses.hasNext()) { + final TaskStatusPlus status = taskStatuses.next(); + + // taskType can be null if middleManagers are running with an older version. Here, we consevatively regard + // the tasks of the unknown taskType as the compactionTask. This is because it's important to not run + // compactionTasks more than the configured limit at any time which might impact to the ingestion + // performance. + if (status.getType() == null || COMPACTION_TASK_TYPE.equals(status.getType())) { + retVal.add(status); + } + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + + return retVal; } private int getCompactionTaskCapacity(CoordinatorCompactionConfig dynamicConfig) { int totalWorkerCapacity; try { - totalWorkerCapacity = dynamicConfig.isUseAutoScaleSlots() - ? indexingServiceClient.getTotalWorkerCapacityWithAutoScale() - : indexingServiceClient.getTotalWorkerCapacity(); + final IndexingTotalWorkerCapacityInfo workerCapacityInfo = + FutureUtils.get(overlordClient.getTotalWorkerCapacity(), true); + + if (dynamicConfig.isUseAutoScaleSlots() && workerCapacityInfo.getMaximumCapacityWithAutoScale() > 0) { + totalWorkerCapacity = workerCapacityInfo.getMaximumCapacityWithAutoScale(); + } else { + totalWorkerCapacity = workerCapacityInfo.getCurrentClusterCapacity(); + } } - catch (Exception e) { - LOG.warn("Failed to get total worker capacity with auto scale slots. Falling back to current capacity count"); - totalWorkerCapacity = indexingServiceClient.getTotalWorkerCapacity(); + catch (ExecutionException e) { + // Call to getTotalWorkerCapacity may fail during a rolling upgrade: API was added in 0.23.0. + if (e.getCause() instanceof HttpResponseException + && ((HttpResponseException) e.getCause()).getResponse().getStatus().equals(HttpResponseStatus.NOT_FOUND)) { + LOG.noStackTrace().warn(e, "Call to getTotalWorkerCapacity failed. Falling back to getWorkers."); + totalWorkerCapacity = + FutureUtils.getUnchecked(overlordClient.getWorkers(), true) + .stream() + .mapToInt(worker -> worker.getWorker().getCapacity()) + .sum(); + } else { + throw new RuntimeException(e.getCause()); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); } return Math.min( @@ -476,7 +533,7 @@ public class CompactSegments implements CoordinatorCustomDuty } } - final String taskId = indexingServiceClient.compactSegments( + final String taskId = compactSegments( "coordinator-issued", segmentsToCompact, config.getTaskPriority(), @@ -627,4 +684,48 @@ public class CompactSegments implements CoordinatorCustomDuty { return autoCompactionSnapshotPerDataSource.get(); } + + private 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 + ) + { + Preconditions.checkArgument(!segments.isEmpty(), "Expect non-empty segments to compact"); + + final String dataSource = segments.get(0).getDataSource(); + Preconditions.checkArgument( + segments.stream().allMatch(segment -> segment.getDataSource().equals(dataSource)), + "Segments must have the same dataSource" + ); + + context = context == null ? new HashMap<>() : context; + context.put("priority", compactionTaskPriority); + + final String taskId = IdUtils.newTaskId(idPrefix, ClientCompactionTaskQuery.TYPE, dataSource, null); + final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); + final ClientTaskQuery taskPayload = new ClientCompactionTaskQuery( + taskId, + dataSource, + new ClientCompactionIOConfig( + ClientCompactionIntervalSpec.fromSegments(segments, segmentGranularity), + dropExisting + ), + tuningConfig, + granularitySpec, + dimensionsSpec, + metricsSpec, + transformSpec, + context + ); + FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); + return taskId; + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java index f1659b33350..f4b6240b43a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java @@ -21,11 +21,12 @@ package org.apache.druid.server.coordinator.duty; import com.google.common.base.Preconditions; import com.google.inject.Inject; -import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.utils.CollectionUtils; @@ -55,12 +56,12 @@ public class KillUnusedSegments implements CoordinatorDuty private long lastKillTime = 0; private final SegmentsMetadataManager segmentsMetadataManager; - private final IndexingServiceClient indexingServiceClient; + private final OverlordClient overlordClient; @Inject public KillUnusedSegments( SegmentsMetadataManager segmentsMetadataManager, - IndexingServiceClient indexingServiceClient, + OverlordClient overlordClient, DruidCoordinatorConfig config ) { @@ -91,7 +92,7 @@ public class KillUnusedSegments implements CoordinatorDuty ); this.segmentsMetadataManager = segmentsMetadataManager; - this.indexingServiceClient = indexingServiceClient; + this.overlordClient = overlordClient; } @Override @@ -129,7 +130,7 @@ public class KillUnusedSegments implements CoordinatorDuty } try { - indexingServiceClient.killUnusedSegments("coordinator-issued", dataSource, intervalToKill); + FutureUtils.getUnchecked(overlordClient.runKillTask("coordinator-issued", dataSource, intervalToKill), true); ++submittedTasks; } catch (Exception ex) { diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 291bfbc8fa0..4a00f61300c 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -36,7 +36,7 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.client.SegmentLoadInfo; -import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -50,6 +50,7 @@ import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.metadata.UnknownSegmentIdsException; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.rules.LoadRule; @@ -104,7 +105,7 @@ public class DataSourcesResource private final CoordinatorServerView serverInventoryView; private final SegmentsMetadataManager segmentsMetadataManager; private final MetadataRuleManager metadataRuleManager; - private final IndexingServiceClient indexingServiceClient; + private final OverlordClient overlordClient; private final AuthorizerMapper authorizerMapper; private final DruidCoordinator coordinator; @@ -113,7 +114,7 @@ public class DataSourcesResource CoordinatorServerView serverInventoryView, SegmentsMetadataManager segmentsMetadataManager, MetadataRuleManager metadataRuleManager, - @Nullable IndexingServiceClient indexingServiceClient, + @Nullable OverlordClient overlordClient, AuthorizerMapper authorizerMapper, DruidCoordinator coordinator ) @@ -121,7 +122,7 @@ public class DataSourcesResource this.serverInventoryView = serverInventoryView; this.segmentsMetadataManager = segmentsMetadataManager; this.metadataRuleManager = metadataRuleManager; - this.indexingServiceClient = indexingServiceClient; + this.overlordClient = overlordClient; this.authorizerMapper = authorizerMapper; this.coordinator = coordinator; } @@ -311,7 +312,7 @@ public class DataSourcesResource @QueryParam("interval") final String interval ) { - if (indexingServiceClient == null) { + if (overlordClient == null) { return Response.ok(ImmutableMap.of("error", "no indexing service found")).build(); } @@ -333,7 +334,7 @@ public class DataSourcesResource @PathParam("interval") final String interval ) { - if (indexingServiceClient == null) { + if (overlordClient == null) { return Response.ok(ImmutableMap.of("error", "no indexing service found")).build(); } if (StringUtils.contains(interval, '_')) { @@ -341,7 +342,7 @@ public class DataSourcesResource } final Interval theInterval = Intervals.of(interval.replace('_', '/')); try { - indexingServiceClient.killUnusedSegments("api-issued", dataSourceName, theInterval); + FutureUtils.getUnchecked(overlordClient.runKillTask("api-issued", dataSourceName, theInterval), true); return Response.ok().build(); } catch (Exception e) { diff --git a/server/src/main/java/org/apache/druid/server/http/OverlordProxyServlet.java b/server/src/main/java/org/apache/druid/server/http/OverlordProxyServlet.java index 0fa0f3e2c64..a43a5ce379d 100644 --- a/server/src/main/java/org/apache/druid/server/http/OverlordProxyServlet.java +++ b/server/src/main/java/org/apache/druid/server/http/OverlordProxyServlet.java @@ -21,11 +21,10 @@ package org.apache.druid.server.http; import com.google.inject.Inject; import com.google.inject.Provider; -import org.apache.druid.client.indexing.IndexingService; -import org.apache.druid.discovery.DruidLeaderClient; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.guice.annotations.Global; import org.apache.druid.guice.http.DruidHttpClientConfig; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.JettyUtils; import org.apache.druid.server.security.AuthConfig; import org.eclipse.jetty.client.HttpClient; @@ -41,18 +40,18 @@ import javax.servlet.http.HttpServletResponse; */ public class OverlordProxyServlet extends ProxyServlet { - private final DruidLeaderClient druidLeaderClient; + private final OverlordClient overlordClient; private final Provider httpClientProvider; private final DruidHttpClientConfig httpClientConfig; @Inject OverlordProxyServlet( - @IndexingService DruidLeaderClient druidLeaderClient, + OverlordClient overlordClient, @Global Provider httpClientProvider, @Global DruidHttpClientConfig httpClientConfig ) { - this.druidLeaderClient = druidLeaderClient; + this.overlordClient = overlordClient; this.httpClientProvider = httpClientProvider; this.httpClientConfig = httpClientConfig; } @@ -60,10 +59,7 @@ public class OverlordProxyServlet extends ProxyServlet @Override protected String rewriteTarget(HttpServletRequest request) { - final String overlordLeader = druidLeaderClient.findCurrentLeader(); - if (overlordLeader == null) { - throw new ISE("Can't find Overlord leader."); - } + final String overlordLeader = FutureUtils.getUnchecked(overlordClient.findCurrentLeader(), true).toString(); return JettyUtils.concatenateForRewrite( overlordLeader, diff --git a/server/src/test/java/org/apache/druid/client/indexing/HttpIndexingServiceClientTest.java b/server/src/test/java/org/apache/druid/client/indexing/HttpIndexingServiceClientTest.java deleted file mode 100644 index f669b4a8d87..00000000000 --- a/server/src/test/java/org/apache/druid/client/indexing/HttpIndexingServiceClientTest.java +++ /dev/null @@ -1,469 +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 com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.discovery.DruidLeaderClient; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; -import org.apache.druid.segment.NestedDataDimensionSchema; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NoneShardSpec; -import org.easymock.Capture; -import org.easymock.EasyMock; -import org.jboss.netty.buffer.BigEndianHeapChannelBuffer; -import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.buffer.ChannelBuffers; -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.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.util.Map; - -public class HttpIndexingServiceClientTest -{ - private HttpIndexingServiceClient httpIndexingServiceClient; - private ObjectMapper jsonMapper; - private DruidLeaderClient druidLeaderClient; - private ObjectMapper mockMapper; - - @Rule - public ExpectedException expectedException = ExpectedException.none(); - - @Before - public void setup() - { - jsonMapper = new DefaultObjectMapper(); - druidLeaderClient = EasyMock.createMock(DruidLeaderClient.class); - mockMapper = EasyMock.createMock(ObjectMapper.class); - - httpIndexingServiceClient = new HttpIndexingServiceClient( - jsonMapper, - druidLeaderClient - ); - } - - @Test - public void testSample() throws Exception - { - final SamplerResponse samplerResponse = new SamplerResponse( - 2, - 2, - ImmutableList.of( - new StringDimensionSchema("x"), - new StringDimensionSchema("y") - ), - ImmutableList.of( - new NestedDataDimensionSchema("x"), - new NestedDataDimensionSchema("y") - ), - RowSignature.builder().addTimeColumn().add("x", ColumnType.STRING).add("y", ColumnType.STRING).build(), - ImmutableList.of( - new SamplerResponse.SamplerResponseRow( - ImmutableMap.of("time", "2020-01-01", "x", "123", "y", "456"), - ImmutableMap.of("time", "2020-01-01", "x", "123", "y", "456"), - false, - null - ) - ) - ); - - final SamplerSpec samplerSpec = new SamplerSpec() - { - @Override - public SamplerResponse sample() - { - return samplerResponse; - } - }; - - HttpResponse response = EasyMock.createMock(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(jsonMapper.writeValueAsString(samplerResponse)); - - EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class))) - .andReturn(responseHolder) - .anyTimes(); - - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/sampler")) - .andReturn(new Request(HttpMethod.POST, new URL("http://localhost:8090/druid/indexer/v1/sampler"))) - .anyTimes(); - EasyMock.replay(druidLeaderClient); - - final SamplerResponse actualResponse = httpIndexingServiceClient.sample(samplerSpec); - Assert.assertEquals(samplerResponse, actualResponse); - - EasyMock.verify(druidLeaderClient, response); - } - - @Test - public void testSampleError() throws Exception - { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Failed to sample with sampler spec"); - expectedException.expectMessage("Please check overlord log"); - - final SamplerResponse samplerResponse = new SamplerResponse( - 2, - 2, - ImmutableList.of( - new StringDimensionSchema("x"), - new StringDimensionSchema("y") - ), - ImmutableList.of( - new NestedDataDimensionSchema("x"), - new NestedDataDimensionSchema("y") - ), - RowSignature.builder().addTimeColumn().add("x", ColumnType.STRING).add("y", ColumnType.STRING).build(), - ImmutableList.of( - new SamplerResponse.SamplerResponseRow( - ImmutableMap.of("time", "2020-01-01", "x", "123", "y", "456"), - ImmutableMap.of("time", "2020-01-01", "x", "123", "y", "456"), - false, - null - ) - ) - ); - - final SamplerSpec samplerSpec = new SamplerSpec() - { - @Override - public SamplerResponse sample() - { - return samplerResponse; - } - }; - HttpResponse response = EasyMock.createMock(HttpResponse.class); - EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).anyTimes(); - EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0)); - EasyMock.replay(response); - - StringFullResponseHolder responseHolder = new StringFullResponseHolder( - response, - StandardCharsets.UTF_8 - ).addChunk(""); - - EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class))) - .andReturn(responseHolder) - .anyTimes(); - - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/sampler")) - .andReturn(new Request(HttpMethod.POST, new URL("http://localhost:8090/druid/indexer/v1/sampler"))) - .anyTimes(); - EasyMock.replay(druidLeaderClient); - - - httpIndexingServiceClient.sample(samplerSpec); - EasyMock.verify(druidLeaderClient, response); - } - - @Test - public void testGetTaskReport() throws Exception - { - String taskId = "testTaskId"; - HttpResponse response = EasyMock.createMock(HttpResponse.class); - EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0)); - EasyMock.replay(response); - - Map dummyResponse = ImmutableMap.of("test", "value"); - - StringFullResponseHolder responseHolder = new StringFullResponseHolder( - response, - StandardCharsets.UTF_8 - ).addChunk(jsonMapper.writeValueAsString(dummyResponse)); - - EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class))) - .andReturn(responseHolder) - .anyTimes(); - - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/task/testTaskId/reports")) - .andReturn(new Request( - HttpMethod.GET, - new URL("http://localhost:8090/druid/indexer/v1/task/testTaskId/reports") - )) - .anyTimes(); - EasyMock.replay(druidLeaderClient); - - final Map actualResponse = httpIndexingServiceClient.getTaskReport(taskId); - Assert.assertEquals(dummyResponse, actualResponse); - - EasyMock.verify(druidLeaderClient, response); - } - - @Test - public void testGetTaskReportStatusNotFound() throws Exception - { - String taskId = "testTaskId"; - HttpResponse response = EasyMock.createMock(HttpResponse.class); - String errorMsg = "No task reports were found for this task. " - + "The task may not exist, or it may not have completed yet."; - ChannelBuffer buf = ChannelBuffers.buffer(errorMsg.length()); - buf.writeBytes(errorMsg.getBytes(StandardCharsets.UTF_8)); - - EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); - EasyMock.expect(response.getContent()).andReturn(buf); - EasyMock.replay(response); - - StringFullResponseHolder responseHolder = new StringFullResponseHolder( - response, - StandardCharsets.UTF_8 - ).addChunk(""); - - EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class))) - .andReturn(responseHolder) - .anyTimes(); - - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/task/testTaskId/reports")) - .andReturn(new Request( - HttpMethod.GET, - new URL("http://localhost:8090/druid/indexer/v1/task/testTaskId/reports") - )) - .anyTimes(); - EasyMock.replay(druidLeaderClient); - - final Map actualResponse = httpIndexingServiceClient.getTaskReport(taskId); - Assert.assertNull(actualResponse); - - EasyMock.verify(druidLeaderClient, response); - } - - @Test - public void testGetTaskReportEmpty() throws Exception - { - String taskId = "testTaskId"; - HttpResponse response = EasyMock.createMock(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(""); - - EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class))) - .andReturn(responseHolder) - .anyTimes(); - - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/task/testTaskId/reports")) - .andReturn(new Request( - HttpMethod.GET, - new URL("http://localhost:8090/druid/indexer/v1/task/testTaskId/reports") - )) - .anyTimes(); - EasyMock.replay(druidLeaderClient); - - final Map actualResponse = httpIndexingServiceClient.getTaskReport(taskId); - Assert.assertNull(actualResponse); - - EasyMock.verify(druidLeaderClient, response); - } - - @Test - public void testCompact() throws Exception - { - DataSegment segment = new DataSegment( - "test", - Intervals.of("2015-04-12/2015-04-13"), - "1", - ImmutableMap.of("bucket", "bucket", "path", "test/2015-04-12T00:00:00.000Z_2015-04-13T00:00:00.000Z/1/0/index.zip"), - null, - null, - NoneShardSpec.instance(), - 0, - 1 - ); - Capture captureTask = EasyMock.newCapture(); - HttpResponse response = EasyMock.createMock(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(jsonMapper.writeValueAsString(ImmutableMap.of("task", "aaa"))); - - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/task")) - .andReturn(new Request(HttpMethod.POST, new URL("http://localhost:8090/druid/indexer/v1/task"))) - .anyTimes(); - EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class))) - .andReturn(responseHolder) - .anyTimes(); - EasyMock.expect(mockMapper.writeValueAsBytes(EasyMock.capture(captureTask))) - .andReturn(new byte[]{1, 2, 3}) - .anyTimes(); - EasyMock.expect(mockMapper.readValue(EasyMock.anyString(), EasyMock.eq(JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT))) - .andReturn(ImmutableMap.of()) - .anyTimes(); - EasyMock.replay(druidLeaderClient, mockMapper); - - HttpIndexingServiceClient httpIndexingServiceClient = new HttpIndexingServiceClient( - mockMapper, - druidLeaderClient - ); - - try { - httpIndexingServiceClient.compactSegments( - "test-compact", - ImmutableList.of(segment), - 50, - null, - null, - null, - null, - null, - null, - null - ); - } - catch (Exception e) { - // Ignore IllegalStateException as taskId is internally generated and returned task id will failed check - Assert.assertEquals(IllegalStateException.class.getName(), e.getCause().getClass().getName()); - } - ClientCompactionTaskQuery taskQuery = (ClientCompactionTaskQuery) captureTask.getValue(); - Assert.assertEquals(Intervals.of("2015-04-12/2015-04-13"), taskQuery.getIoConfig().getInputSpec().getInterval()); - Assert.assertNull(taskQuery.getGranularitySpec()); - Assert.assertNull(taskQuery.getIoConfig().getInputSpec().getSha256OfSortedSegmentIds()); - } - - @Test - public void testCompactWithSegmentGranularity() throws Exception - { - DataSegment segment = new DataSegment( - "test", - Intervals.of("2015-04-12/2015-04-13"), - "1", - ImmutableMap.of("bucket", "bucket", "path", "test/2015-04-12T00:00:00.000Z_2015-04-13T00:00:00.000Z/1/0/index.zip"), - null, - null, - NoneShardSpec.instance(), - 0, - 1 - ); - Capture captureTask = EasyMock.newCapture(); - HttpResponse response = EasyMock.createMock(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(jsonMapper.writeValueAsString(ImmutableMap.of("task", "aaa"))); - - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/task")) - .andReturn(new Request(HttpMethod.POST, new URL("http://localhost:8090/druid/indexer/v1/task"))) - .anyTimes(); - EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class))) - .andReturn(responseHolder) - .anyTimes(); - EasyMock.expect(mockMapper.writeValueAsBytes(EasyMock.capture(captureTask))) - .andReturn(new byte[]{1, 2, 3}) - .anyTimes(); - EasyMock.expect(mockMapper.readValue(EasyMock.anyString(), EasyMock.eq(JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT))) - .andReturn(ImmutableMap.of()) - .anyTimes(); - EasyMock.replay(druidLeaderClient, mockMapper); - - HttpIndexingServiceClient httpIndexingServiceClient = new HttpIndexingServiceClient( - mockMapper, - druidLeaderClient - ); - - try { - httpIndexingServiceClient.compactSegments( - "test-compact", - ImmutableList.of(segment), - 50, - null, - new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null, null), - null, - null, - null, - null, - null - ); - } - catch (Exception e) { - // Ignore IllegalStateException as taskId is internally generated and returned task id will failed check - Assert.assertEquals(IllegalStateException.class.getName(), e.getCause().getClass().getName()); - } - ClientCompactionTaskQuery taskQuery = (ClientCompactionTaskQuery) captureTask.getValue(); - Assert.assertEquals(Intervals.of("2015-01-01/2016-01-01"), taskQuery.getIoConfig().getInputSpec().getInterval()); - Assert.assertEquals(Granularities.YEAR, taskQuery.getGranularitySpec().getSegmentGranularity()); - Assert.assertNull(taskQuery.getIoConfig().getInputSpec().getSha256OfSortedSegmentIds()); - } - - @Test - public void testGetTotalWorkerCapacityWithAutoScale() throws Exception - { - int currentClusterCapacity = 5; - int maximumCapacityWithAutoScale = 10; - // Mock response for /druid/indexer/v1/totalWorkerCapacity - HttpResponse totalWorkerCapacityResponse = EasyMock.createMock(HttpResponse.class); - EasyMock.expect(totalWorkerCapacityResponse.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(totalWorkerCapacityResponse.getContent()).andReturn(new BigEndianHeapChannelBuffer(0)); - EasyMock.replay(totalWorkerCapacityResponse); - IndexingTotalWorkerCapacityInfo indexingTotalWorkerCapacityInfo = new IndexingTotalWorkerCapacityInfo(currentClusterCapacity, maximumCapacityWithAutoScale); - StringFullResponseHolder autoScaleResponseHolder = new StringFullResponseHolder( - totalWorkerCapacityResponse, - StandardCharsets.UTF_8 - ).addChunk(jsonMapper.writeValueAsString(indexingTotalWorkerCapacityInfo)); - EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class))) - .andReturn(autoScaleResponseHolder) - .once(); - EasyMock.expect(druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/totalWorkerCapacity")) - .andReturn(new Request( - HttpMethod.GET, - new URL("http://localhost:8090/druid/indexer/v1/totalWorkerCapacity") - )) - .once(); - EasyMock.replay(druidLeaderClient); - - final int actualResponse = httpIndexingServiceClient.getTotalWorkerCapacityWithAutoScale(); - Assert.assertEquals(maximumCapacityWithAutoScale, actualResponse); - EasyMock.verify(druidLeaderClient); - } -} - 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 index dbfa1ec1047..6882c6b762d 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java @@ -21,14 +21,27 @@ package org.apache.druid.client.indexing; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; +import org.joda.time.Interval; +import javax.annotation.Nullable; +import java.net.URI; +import java.util.List; import java.util.Map; import java.util.Set; public class NoopOverlordClient implements OverlordClient { + @Override + public ListenableFuture findCurrentLeader() + { + throw new UnsupportedOperationException(); + } + @Override public ListenableFuture runTask(String taskId, Object taskObject) { @@ -41,6 +54,16 @@ public class NoopOverlordClient implements OverlordClient throw new UnsupportedOperationException(); } + @Override + public ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ) + { + throw new UnsupportedOperationException(); + } + @Override public ListenableFuture> taskStatuses(Set taskIds) { @@ -53,6 +76,12 @@ public class NoopOverlordClient implements OverlordClient throw new UnsupportedOperationException(); } + @Override + public ListenableFuture taskPayload(String taskId) + { + throw new UnsupportedOperationException(); + } + @Override public ListenableFuture> taskReportAsMap(String taskId) { @@ -60,7 +89,31 @@ public class NoopOverlordClient implements OverlordClient } @Override - public ListenableFuture taskPayload(String taskId) + public ListenableFuture> supervisorStatuses() + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture>> findLockedIntervals(Map minTaskPriority) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture killPendingSegments(String dataSource, Interval interval) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture> getWorkers() + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture getTotalWorkerCapacity() { throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceSelectorConfig.java b/server/src/test/java/org/apache/druid/indexing/IndexingWorkerInfoTest.java similarity index 70% rename from server/src/main/java/org/apache/druid/client/indexing/IndexingServiceSelectorConfig.java rename to server/src/test/java/org/apache/druid/indexing/IndexingWorkerInfoTest.java index 0189fd622b5..d0dd8437384 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceSelectorConfig.java +++ b/server/src/test/java/org/apache/druid/indexing/IndexingWorkerInfoTest.java @@ -17,21 +17,17 @@ * under the License. */ -package org.apache.druid.client.indexing; +package org.apache.druid.indexing; -import com.fasterxml.jackson.annotation.JsonProperty; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.client.indexing.IndexingWorkerInfo; +import org.junit.Test; -/** - */ -public class IndexingServiceSelectorConfig +public class IndexingWorkerInfoTest { - public static final String DEFAULT_SERVICE_NAME = "druid/overlord"; - - @JsonProperty - private String serviceName = DEFAULT_SERVICE_NAME; - - public String getServiceName() + @Test + public void testEquals() { - return serviceName; + EqualsVerifier.forClass(IndexingWorkerInfo.class).usingGetClass().verify(); } } diff --git a/server/src/test/java/org/apache/druid/indexing/IndexingWorkerTest.java b/server/src/test/java/org/apache/druid/indexing/IndexingWorkerTest.java new file mode 100644 index 00000000000..7cb423c599d --- /dev/null +++ b/server/src/test/java/org/apache/druid/indexing/IndexingWorkerTest.java @@ -0,0 +1,33 @@ +/* + * 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; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.client.indexing.IndexingWorker; +import org.junit.Test; + +public class IndexingWorkerTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(IndexingWorker.class).usingGetClass().verify(); + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java b/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java index 60e6a347048..da817c3da3b 100644 --- a/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java +++ b/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java @@ -71,13 +71,13 @@ public class MockServiceClient implements ServiceClient return this; } - public MockServiceClient expect(final RequestBuilder request, final HttpResponse response) + public MockServiceClient expectAndRespond(final RequestBuilder request, final HttpResponse response) { expectations.add(new Expectation(request, Either.value(response))); return this; } - public MockServiceClient expect( + public MockServiceClient expectAndRespond( final RequestBuilder request, final HttpResponseStatus status, final Map headers, @@ -91,10 +91,10 @@ public class MockServiceClient implements ServiceClient if (content != null) { response.setContent(ChannelBuffers.wrappedBuffer(content)); } - return expect(request, response); + return expectAndRespond(request, response); } - public MockServiceClient expect(final RequestBuilder request, final Throwable e) + public MockServiceClient expectAndThrow(final RequestBuilder request, final Throwable e) { expectations.add(new Expectation(request, Either.error(e))); return this; diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java index 26fb972ac5c..2d89bded802 100644 --- a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -20,39 +20,420 @@ package org.apache.druid.rpc.indexing; 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 com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery; import org.apache.druid.client.indexing.ClientTaskQuery; +import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; +import org.apache.druid.client.indexing.IndexingWorker; +import org.apache.druid.client.indexing.IndexingWorkerInfo; import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.indexer.RunnerTaskState; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.jboss.netty.handler.codec.http.DefaultHttpResponse; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.jboss.netty.handler.codec.http.HttpVersion; +import org.joda.time.Interval; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; +import javax.ws.rs.core.HttpHeaders; +import javax.ws.rs.core.MediaType; +import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutionException; public class OverlordClientImplTest { + private static final List STATUSES = Collections.singletonList( + new TaskStatusPlus( + "taskId", + null, + null, + DateTimes.of("2000"), + DateTimes.of("2000"), + TaskState.RUNNING, + RunnerTaskState.RUNNING, + null, + TaskLocation.unknown(), + null, + null + ) + ); + private ObjectMapper jsonMapper; + private MockServiceClient serviceClient; + private OverlordClient overlordClient; + + @Before + public void setup() + { + jsonMapper = new DefaultObjectMapper(); + serviceClient = new MockServiceClient(); + overlordClient = new OverlordClientImpl(serviceClient, jsonMapper); + } + + @After + public void tearDown() + { + serviceClient.verify(); + } @Test - public void testTaskPayload() throws ExecutionException, InterruptedException, JsonProcessingException + public void test_findCurrentLeader() throws Exception { - final String taskID = "taskId_1"; - MockServiceClient client = new MockServiceClient(); - final OverlordClientImpl overlordClient = new OverlordClientImpl(client, DefaultObjectMapper.INSTANCE); - - ClientTaskQuery clientTaskQuery = new ClientKillUnusedSegmentsTaskQuery(taskID, "test", null, null); - - client.expect(new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/task/" + taskID), - HttpResponseStatus.OK, - Collections.emptyMap(), - DefaultObjectMapper.INSTANCE.writeValueAsBytes(new TaskPayloadResponse(taskID, clientTaskQuery)) + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/leader"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + StringUtils.toUtf8("http://followTheLeader") ); - Assert.assertEquals(clientTaskQuery, overlordClient.taskPayload(taskID).get().getPayload()); + Assert.assertEquals(URI.create("http://followTheLeader"), overlordClient.findCurrentLeader().get()); + } + + @Test + public void test_taskStatuses_null_null_null() throws Exception + { + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/tasks"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(STATUSES) + ); + + Assert.assertEquals( + STATUSES, + ImmutableList.copyOf(overlordClient.taskStatuses(null, null, null).get()) + ); + } + + @Test + public void test_taskStatuses_RUNNING_null_null() throws Exception + { + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/tasks?state=RUNNING"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(STATUSES) + ); + + Assert.assertEquals( + STATUSES, + ImmutableList.copyOf(overlordClient.taskStatuses("RUNNING", null, null).get()) + ); + } + + @Test + public void test_taskStatuses_RUNNING_foo_null() throws Exception + { + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/tasks?state=RUNNING&datasource=foo"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(STATUSES) + ); + + Assert.assertEquals( + STATUSES, + ImmutableList.copyOf(overlordClient.taskStatuses("RUNNING", "foo", null).get()) + ); + } + + @Test + public void test_taskStatuses_null_foo_null() throws Exception + { + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/tasks?datasource=foo"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(STATUSES) + ); + + Assert.assertEquals( + STATUSES, + ImmutableList.copyOf(overlordClient.taskStatuses(null, "foo", null).get()) + ); + } + + @Test + public void test_taskStatuses_RUNNING_foo_zero() throws Exception + { + serviceClient.expectAndRespond( + new RequestBuilder( + HttpMethod.GET, + "/druid/indexer/v1/tasks?state=RUNNING&datasource=foo%3F&maxCompletedTasks=0" + ), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(STATUSES) + ); + + Assert.assertEquals( + STATUSES, + ImmutableList.copyOf(overlordClient.taskStatuses("RUNNING", "foo?", 0).get()) + ); + } + + @Test + public void test_taskStatuses_null_null_zero() throws Exception + { + serviceClient.expectAndRespond( + new RequestBuilder( + HttpMethod.GET, + "/druid/indexer/v1/tasks?maxCompletedTasks=0" + ), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(STATUSES) + ); + + Assert.assertEquals( + STATUSES, + ImmutableList.copyOf(overlordClient.taskStatuses(null, null, 0).get()) + ); + } + + @Test + public void test_findLockedIntervals() throws Exception + { + final Map priorityMap = ImmutableMap.of("foo", 3); + final Map> lockMap = + ImmutableMap.of("foo", Collections.singletonList(Intervals.of("2000/2001"))); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/lockedIntervals") + .jsonContent(jsonMapper, priorityMap), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(lockMap) + ); + + Assert.assertEquals( + lockMap, + overlordClient.findLockedIntervals(priorityMap).get() + ); + } + + @Test + public void test_findLockedIntervals_nullReturn() throws Exception + { + final Map priorityMap = ImmutableMap.of("foo", 3); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/lockedIntervals") + .jsonContent(jsonMapper, priorityMap), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(null) + ); + + Assert.assertEquals( + Collections.emptyMap(), + overlordClient.findLockedIntervals(priorityMap).get() + ); + } + + @Test + public void test_supervisorStatuses() throws Exception + { + final List statuses = ImmutableList.of( + new SupervisorStatus.Builder() + .withId("foo") + .withSource("kafka") + .withState("chill") + .build() + ); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/supervisor?system"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(statuses) + ); + + Assert.assertEquals( + statuses, + ImmutableList.copyOf(overlordClient.supervisorStatuses().get()) + ); + } + + @Test + public void test_taskReportAsMap() throws Exception + { + final String taskId = "testTaskId"; + final Map response = ImmutableMap.of("test", "value"); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/task/testTaskId/reports"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(response) + ); + + final ListenableFuture> future = overlordClient.taskReportAsMap(taskId); + Assert.assertEquals(response, future.get()); + } + + @Test + public void test_taskReportAsMap_notFound() + { + String taskId = "testTaskId"; + String errorMsg = "No task reports were found for this task. " + + "The task may not exist, or it may not have completed yet."; + + serviceClient.expectAndThrow( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/task/testTaskId/reports"), + new HttpResponseException( + new StringFullResponseHolder( + new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NOT_FOUND), + StandardCharsets.UTF_8 + ).addChunk(errorMsg) + ) + ); + + final ListenableFuture> future = overlordClient.taskReportAsMap(taskId); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + future::get + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(HttpResponseException.class)); + Assert.assertEquals( + HttpResponseStatus.NOT_FOUND.getCode(), + ((HttpResponseException) e.getCause()).getResponse().getStatus().getCode() + ); + } + + @Test + public void test_getTaskReport_empty() + { + final String taskID = "testTaskId"; + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/task/" + taskID + "/reports"), + HttpResponseStatus.OK, + Collections.emptyMap(), + StringUtils.toUtf8("{}") + ); + + final Map actualResponse = + FutureUtils.getUnchecked(overlordClient.taskReportAsMap(taskID), true); + Assert.assertEquals(Collections.emptyMap(), actualResponse); + } + + @Test + public void test_getTotalWorkerCapacity() throws Exception + { + int currentClusterCapacity = 5; + int maximumCapacityWithAutoScale = 10; + + // Mock response for /druid/indexer/v1/totalWorkerCapacity + final IndexingTotalWorkerCapacityInfo indexingTotalWorkerCapacityInfo = new IndexingTotalWorkerCapacityInfo( + currentClusterCapacity, + maximumCapacityWithAutoScale + ); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/totalWorkerCapacity"), + HttpResponseStatus.OK, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(indexingTotalWorkerCapacityInfo) + ); + + Assert.assertEquals( + indexingTotalWorkerCapacityInfo, + FutureUtils.getUnchecked(overlordClient.getTotalWorkerCapacity(), true) + ); + } + + @Test + public void test_getWorkers() throws Exception + { + final List workers = ImmutableList.of( + new IndexingWorkerInfo( + new IndexingWorker("http", "localhost", "1.2.3.4", 3, "2"), + 0, + Collections.emptySet(), + Collections.emptyList(), + DateTimes.of("2000"), + null + ) + ); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/workers"), + HttpResponseStatus.OK, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(workers) + ); + + Assert.assertEquals( + workers, + FutureUtils.getUnchecked(overlordClient.getWorkers(), true) + ); + } + + @Test + public void test_killPendingSegments() throws Exception + { + serviceClient.expectAndRespond( + new RequestBuilder( + HttpMethod.DELETE, + "/druid/indexer/v1/pendingSegments/foo?interval=2000-01-01T00%3A00%3A00.000Z%2F2001-01-01T00%3A00%3A00.000Z" + ), + HttpResponseStatus.OK, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes( + ImmutableMap.of("numDeleted", 2L) + ) + ); + + Assert.assertEquals( + Integer.valueOf(2), + FutureUtils.getUnchecked(overlordClient.killPendingSegments("foo", Intervals.of("2000/2001")), true) + ); + } + + @Test + public void test_taskPayload() throws ExecutionException, InterruptedException, JsonProcessingException + { + final String taskID = "taskId_1"; + final ClientTaskQuery clientTaskQuery = new ClientKillUnusedSegmentsTaskQuery(taskID, "test", null, null); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/task/" + taskID), + HttpResponseStatus.OK, + Collections.emptyMap(), + DefaultObjectMapper.INSTANCE.writeValueAsBytes(new TaskPayloadResponse(taskID, clientTaskQuery)) + ); + + Assert.assertEquals( + clientTaskQuery, + overlordClient.taskPayload(taskID).get().getPayload() + ); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 7487d332d1f..e9a201c90d1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -27,29 +27,21 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import junitparams.converters.Nullable; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.indexing.ClientCompactionIOConfig; import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; -import org.apache.druid.client.indexing.ClientCompactionTaskDimensionsSpec; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; 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.HttpIndexingServiceClient; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; -import org.apache.druid.client.indexing.IndexingWorker; -import org.apache.druid.client.indexing.IndexingWorkerInfo; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.discovery.DruidLeaderClient; -import org.apache.druid.discovery.DruidNodeDiscovery; -import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeRole; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -59,19 +51,20 @@ 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.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.apache.druid.java.util.common.parsers.CloseableIterator; 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.incremental.OnheapIncrementalIndex; +import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -95,12 +88,6 @@ import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.utils.Streams; -import org.easymock.EasyMock; -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.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -113,9 +100,8 @@ import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; -import java.io.IOException; -import java.net.URL; -import java.nio.charset.StandardCharsets; +import javax.annotation.Nullable; +import java.net.URI; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -124,7 +110,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; -import java.util.function.BooleanSupplier; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -244,30 +229,29 @@ public class CompactSegmentsTest @Test public void testSerde() throws Exception { - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); JSON_MAPPER.setInjectableValues( new InjectableValues.Std() .addValue(DruidCoordinatorConfig.class, COORDINATOR_CONFIG) - .addValue(IndexingServiceClient.class, indexingServiceClient) + .addValue(OverlordClient.class, overlordClient) .addValue(CompactionSegmentSearchPolicy.class, SEARCH_POLICY) ); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); String compactSegmentString = JSON_MAPPER.writeValueAsString(compactSegments); CompactSegments serdeCompactSegments = JSON_MAPPER.readValue(compactSegmentString, CompactSegments.class); Assert.assertNotNull(serdeCompactSegments); + Assert.assertEquals(COORDINATOR_CONFIG.getCompactionSkipLockedIntervals(), serdeCompactSegments.isSkipLockedIntervals()); + Assert.assertSame(overlordClient, serdeCompactSegments.getOverlordClient()); } @Test public void testRun() { - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); final Supplier expectedVersionSupplier = new Supplier() { @@ -342,10 +326,8 @@ public class CompactSegmentsTest @Test public void testMakeStats() { - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -438,10 +420,8 @@ public class CompactSegmentsTest dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -502,10 +482,8 @@ public class CompactSegmentsTest @Test public void testMakeStatsWithDeactivatedDatasource() { - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -596,10 +574,8 @@ public class CompactSegmentsTest dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -657,10 +633,8 @@ public class CompactSegmentsTest @Test public void testRunMultipleCompactionTaskSlots() { - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, 3); Assert.assertEquals(3, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -673,11 +647,10 @@ public class CompactSegmentsTest { int maxCompactionSlot = 3; Assert.assertTrue(maxCompactionSlot < MAXIMUM_CAPACITY_WITH_AUTO_SCALE); - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); - final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); + final CoordinatorRunStats stats = + doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.MAX_SLOTS)); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.SUBMITTED_TASKS)); @@ -688,11 +661,10 @@ public class CompactSegmentsTest { int maxCompactionSlot = 100; Assert.assertFalse(maxCompactionSlot < MAXIMUM_CAPACITY_WITH_AUTO_SCALE); - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); - final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); + final CoordinatorRunStats stats = + doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.MAX_SLOTS)); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.SUBMITTED_TASKS)); @@ -701,8 +673,9 @@ public class CompactSegmentsTest @Test public void testCompactWithoutGranularitySpec() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -742,33 +715,24 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor> segmentsCaptor = ArgumentCaptor.forClass(List.class); - ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskGranularitySpec.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - segmentsCaptor.capture(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - granularitySpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() + + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + + Assert.assertEquals( + Intervals.of("2017-01-09T12:00:00.000Z/2017-01-10T00:00:00.000Z"), + taskPayload.getIoConfig().getInputSpec().getInterval() ); - // Only the same amount of segments as the original PARTITION_PER_TIME_INTERVAL since segment granulartity is the same - Assert.assertEquals(PARTITION_PER_TIME_INTERVAL, segmentsCaptor.getValue().size()); - Assert.assertNull(granularitySpecArgumentCaptor.getValue().getSegmentGranularity()); - Assert.assertNull(granularitySpecArgumentCaptor.getValue().getQueryGranularity()); - Assert.assertNull(granularitySpecArgumentCaptor.getValue().isRollup()); + Assert.assertNull(taskPayload.getGranularitySpec().getSegmentGranularity()); + Assert.assertNull(taskPayload.getGranularitySpec().getQueryGranularity()); + Assert.assertNull(taskPayload.getGranularitySpec().isRollup()); } @Test public void testCompactWithNotNullIOConfig() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -808,27 +772,16 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor dropExistingCapture = ArgumentCaptor.forClass(Boolean.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - dropExistingCapture.capture(), - ArgumentMatchers.any() - ); - Assert.assertEquals(true, dropExistingCapture.getValue()); + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + Assert.assertTrue(taskPayload.getIoConfig().isDropExisting()); } @Test public void testCompactWithNullIOConfig() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -868,27 +821,16 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor dropExistingCapture = ArgumentCaptor.forClass(Boolean.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - dropExistingCapture.capture(), - ArgumentMatchers.any() - ); - Assert.assertNull(dropExistingCapture.getValue()); + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + Assert.assertEquals(BatchIOConfig.DEFAULT_DROP_EXISTING, taskPayload.getIoConfig().isDropExisting()); } @Test public void testCompactWithGranularitySpec() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -928,35 +870,27 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor> segmentsCaptor = ArgumentCaptor.forClass(List.class); - ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskGranularitySpec.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - segmentsCaptor.capture(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - granularitySpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year - Assert.assertEquals(datasourceToSegments.get(dataSource).size(), segmentsCaptor.getValue().size()); - ClientCompactionTaskGranularitySpec actual = granularitySpecArgumentCaptor.getValue(); - Assert.assertNotNull(actual); - ClientCompactionTaskGranularitySpec expected = new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null, null); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), + taskPayload.getIoConfig().getInputSpec() + ); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = + new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null, null); + Assert.assertEquals(expectedGranularitySpec, taskPayload.getGranularitySpec()); } @Test public void testCompactWithDimensionSpec() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -996,30 +930,19 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor dimensionsSpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskDimensionsSpec.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - dimensionsSpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + Assert.assertEquals( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")), + taskPayload.getDimensionsSpec().getDimensions() ); - ClientCompactionTaskDimensionsSpec actual = dimensionsSpecArgumentCaptor.getValue(); - Assert.assertNotNull(actual); - Assert.assertEquals(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")), actual.getDimensions()); } @Test public void testCompactWithoutDimensionSpec() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1059,29 +982,16 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor dimensionsSpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskDimensionsSpec.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - dimensionsSpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); - ClientCompactionTaskDimensionsSpec actual = dimensionsSpecArgumentCaptor.getValue(); - Assert.assertNull(actual); + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + Assert.assertNull(taskPayload.getDimensionsSpec()); } @Test public void testCompactWithRollupInGranularitySpec() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1121,26 +1031,18 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor> segmentsCaptor = ArgumentCaptor.forClass(List.class); - ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskGranularitySpec.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - segmentsCaptor.capture(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - granularitySpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + + // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment + // are within the same year + Assert.assertEquals( + ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), + taskPayload.getIoConfig().getInputSpec() ); - Assert.assertEquals(datasourceToSegments.get(dataSource).size(), segmentsCaptor.getValue().size()); - ClientCompactionTaskGranularitySpec actual = granularitySpecArgumentCaptor.getValue(); - Assert.assertNotNull(actual); - ClientCompactionTaskGranularitySpec expected = new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null, true); - Assert.assertEquals(expected, actual); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = + new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null, true); + Assert.assertEquals(expectedGranularitySpec, taskPayload.getGranularitySpec()); } @Test @@ -1148,8 +1050,7 @@ public class CompactSegmentsTest { final String dataSource = DATA_SOURCE_PREFIX + 0; final String conflictTaskId = "taskIdDummy"; - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - TaskStatusPlus runningConflictCompactionTask = new TaskStatusPlus( + final TaskStatusPlus runningConflictCompactionTask = new TaskStatusPlus( conflictTaskId, "groupId", "compact", @@ -1162,7 +1063,7 @@ public class CompactSegmentsTest dataSource, null ); - TaskPayloadResponse runningConflictCompactionTaskPayload = new TaskPayloadResponse( + final TaskPayloadResponse runningConflictCompactionTaskPayload = new TaskPayloadResponse( conflictTaskId, new ClientCompactionTaskQuery( conflictTaskId, @@ -1182,10 +1083,25 @@ public class CompactSegmentsTest null ) ); - Mockito.when(mockIndexingServiceClient.getActiveTasks()).thenReturn(ImmutableList.of(runningConflictCompactionTask)); - Mockito.when(mockIndexingServiceClient.getTaskPayload(ArgumentMatchers.eq(conflictTaskId))).thenReturn(runningConflictCompactionTaskPayload); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = ArgumentCaptor.forClass(Object.class); + Mockito.when(mockClient.runTask(ArgumentMatchers.anyString(), payloadCaptor.capture())) + .thenReturn(Futures.immediateFuture(null)); + Mockito.when(mockClient.taskStatuses(null, null, 0)) + .thenReturn( + Futures.immediateFuture( + CloseableIterators.withEmptyBaggage(ImmutableList.of(runningConflictCompactionTask).iterator()))); + Mockito.when(mockClient.findLockedIntervals(ArgumentMatchers.any())) + .thenReturn(Futures.immediateFuture(Collections.emptyMap())); + Mockito.when(mockClient.cancelTask(conflictTaskId)) + .thenReturn(Futures.immediateFuture(null)); + Mockito.when(mockClient.getTotalWorkerCapacity()) + .thenReturn(Futures.immediateFuture(new IndexingTotalWorkerCapacityInfo(0, 0))); + Mockito.when(mockClient.taskPayload(ArgumentMatchers.eq(conflictTaskId))) + .thenReturn(Futures.immediateFuture(runningConflictCompactionTaskPayload)); + + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( new DataSourceCompactionConfig( @@ -1225,41 +1141,29 @@ public class CompactSegmentsTest ); doCompactSegments(compactSegments, compactionConfigs); // Verify that conflict task was canceled - Mockito.verify(mockIndexingServiceClient).cancelTask(conflictTaskId); + Mockito.verify(mockClient).cancelTask(conflictTaskId); // The active conflict task has interval of 2000/2099 // Make sure that we do not skip interval of conflict task. // Since we cancel the task and will have to compact those intervals with the new segmentGranulartity - ArgumentCaptor> segmentsCaptor = ArgumentCaptor.forClass(List.class); - ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskGranularitySpec.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - segmentsCaptor.capture(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - granularitySpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year - Assert.assertEquals(datasourceToSegments.get(dataSource).size(), segmentsCaptor.getValue().size()); - ClientCompactionTaskGranularitySpec actual = granularitySpecArgumentCaptor.getValue(); - Assert.assertNotNull(actual); - ClientCompactionTaskGranularitySpec expected = new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null, null); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), + taskPayload.getIoConfig().getInputSpec() + ); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = + new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null, null); + Assert.assertEquals(expectedGranularitySpec, taskPayload.getGranularitySpec()); } @Test public void testRunParallelCompactionMultipleCompactionTaskSlots() { - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4); Assert.assertEquals(4, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -1270,35 +1174,34 @@ public class CompactSegmentsTest @Test public void testRunWithLockedIntervals() { - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); // Lock all intervals for dataSource_1 and dataSource_2 final String datasource1 = DATA_SOURCE_PREFIX + 1; - leaderClient.lockedIntervals + overlordClient.lockedIntervals .computeIfAbsent(datasource1, k -> new ArrayList<>()) .add(Intervals.of("2017/2018")); final String datasource2 = DATA_SOURCE_PREFIX + 2; - leaderClient.lockedIntervals + overlordClient.lockedIntervals .computeIfAbsent(datasource2, k -> new ArrayList<>()) .add(Intervals.of("2017/2018")); // Lock all intervals but one for dataSource_0 final String datasource0 = DATA_SOURCE_PREFIX + 0; - leaderClient.lockedIntervals + overlordClient.lockedIntervals .computeIfAbsent(datasource0, k -> new ArrayList<>()) .add(Intervals.of("2017-01-01T13:00:00Z/2017-02-01")); // Verify that locked intervals are skipped and only one compaction task // is submitted for dataSource_0 - CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); - final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4); + CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); + final CoordinatorRunStats stats = + doCompactSegments(compactSegments, createCompactionConfigs(2), 4); Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); - Assert.assertEquals(1, leaderClient.submittedCompactionTasks.size()); + Assert.assertEquals(1, overlordClient.submittedCompactionTasks.size()); - final ClientCompactionTaskQuery compactionTask = leaderClient.submittedCompactionTasks.get(0); + final ClientCompactionTaskQuery compactionTask = overlordClient.submittedCompactionTasks.get(0); Assert.assertEquals(datasource0, compactionTask.getDataSource()); Assert.assertEquals( Intervals.of("2017-01-01T00:00:00/2017-01-01T12:00:00"), @@ -1310,8 +1213,9 @@ public class CompactSegmentsTest public void testCompactWithTransformSpec() { NullHandling.initializeForTests(); - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1351,30 +1255,17 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor transformSpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskTransformSpec.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - transformSpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); - ClientCompactionTaskTransformSpec actual = transformSpecArgumentCaptor.getValue(); - Assert.assertNotNull(actual); - Assert.assertEquals(new SelectorDimFilter("dim1", "foo", null), actual.getFilter()); + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + Assert.assertNotNull(taskPayload.getTransformSpec()); + Assert.assertEquals(new SelectorDimFilter("dim1", "foo", null), taskPayload.getTransformSpec().getFilter()); } @Test public void testCompactWithoutCustomSpecs() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1414,25 +1305,9 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor transformSpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskTransformSpec.class); - ArgumentCaptor metricsSpecArgumentCaptor = ArgumentCaptor.forClass(AggregatorFactory[].class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - metricsSpecArgumentCaptor.capture(), - transformSpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); - ClientCompactionTaskTransformSpec actualTransformSpec = transformSpecArgumentCaptor.getValue(); - Assert.assertNull(actualTransformSpec); - AggregatorFactory[] actualMetricsSpec = metricsSpecArgumentCaptor.getValue(); - Assert.assertNull(actualMetricsSpec); + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + Assert.assertNull(taskPayload.getTransformSpec()); + Assert.assertNull(taskPayload.getMetricsSpec()); } @Test @@ -1440,8 +1315,9 @@ public class CompactSegmentsTest { NullHandling.initializeForTests(); AggregatorFactory[] aggregatorFactories = new AggregatorFactory[] {new CountAggregatorFactory("cnt")}; - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1481,20 +1357,8 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor metricsSpecArgumentCaptor = ArgumentCaptor.forClass(AggregatorFactory[].class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - metricsSpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); - AggregatorFactory[] actual = metricsSpecArgumentCaptor.getValue(); + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + AggregatorFactory[] actual = taskPayload.getMetricsSpec(); Assert.assertNotNull(actual); Assert.assertArrayEquals(aggregatorFactories, actual); } @@ -1504,39 +1368,37 @@ public class CompactSegmentsTest { Mockito.when(COORDINATOR_CONFIG.getCompactionSkipLockedIntervals()).thenReturn(false); - final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); - leaderClient.start(); - HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); // Lock all intervals for all the dataSources final String datasource0 = DATA_SOURCE_PREFIX + 0; - leaderClient.lockedIntervals + overlordClient.lockedIntervals .computeIfAbsent(datasource0, k -> new ArrayList<>()) .add(Intervals.of("2017/2018")); final String datasource1 = DATA_SOURCE_PREFIX + 1; - leaderClient.lockedIntervals + overlordClient.lockedIntervals .computeIfAbsent(datasource1, k -> new ArrayList<>()) .add(Intervals.of("2017/2018")); final String datasource2 = DATA_SOURCE_PREFIX + 2; - leaderClient.lockedIntervals + overlordClient.lockedIntervals .computeIfAbsent(datasource2, k -> new ArrayList<>()) .add(Intervals.of("2017/2018")); // Verify that no locked intervals are skipped - CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); + CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, overlordClient); int maxTaskSlots = partitionsSpec instanceof SingleDimensionPartitionsSpec ? 5 : 3; final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(1), maxTaskSlots); Assert.assertEquals(3, stats.get(Stats.Compaction.SUBMITTED_TASKS)); - Assert.assertEquals(3, leaderClient.submittedCompactionTasks.size()); - leaderClient.submittedCompactionTasks.forEach(task -> { + Assert.assertEquals(3, overlordClient.submittedCompactionTasks.size()); + overlordClient.submittedCompactionTasks.forEach(task -> { System.out.println(task.getDataSource() + " : " + task.getIoConfig().getInputSpec().getInterval()); }); // Verify that tasks are submitted for the latest interval of each dataSource final Map datasourceToInterval = new HashMap<>(); - leaderClient.submittedCompactionTasks.forEach( + overlordClient.submittedCompactionTasks.forEach( task -> datasourceToInterval.put( task.getDataSource(), task.getIoConfig().getInputSpec().getInterval())); Assert.assertEquals( @@ -1586,8 +1448,9 @@ public class CompactSegmentsTest ); dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( new DataSourceCompactionConfig( @@ -1626,26 +1489,16 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor> segmentsCaptor = ArgumentCaptor.forClass(List.class); - ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskGranularitySpec.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - segmentsCaptor.capture(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - granularitySpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + + Assert.assertEquals( + ClientCompactionIntervalSpec.fromSegments(segments, Granularities.DAY), + taskPayload.getIoConfig().getInputSpec() ); - Assert.assertEquals(2, segmentsCaptor.getValue().size()); - ClientCompactionTaskGranularitySpec actual = granularitySpecArgumentCaptor.getValue(); - Assert.assertNotNull(actual); - ClientCompactionTaskGranularitySpec expected = new ClientCompactionTaskGranularitySpec(Granularities.DAY, null, null); - Assert.assertEquals(expected, actual); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = + new ClientCompactionTaskGranularitySpec(Granularities.DAY, null, null); + Assert.assertEquals(expectedGranularitySpec, taskPayload.getGranularitySpec()); } @Test @@ -1681,8 +1534,9 @@ public class CompactSegmentsTest ); dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( new DataSourceCompactionConfig( @@ -1721,33 +1575,24 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor> segmentsCaptor = ArgumentCaptor.forClass(List.class); - ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskGranularitySpec.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - segmentsCaptor.capture(), - ArgumentMatchers.anyInt(), - ArgumentMatchers.any(), - granularitySpecArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + + Assert.assertEquals( + ClientCompactionIntervalSpec.fromSegments(segments, Granularities.YEAR), + taskPayload.getIoConfig().getInputSpec() ); - Assert.assertEquals(2, segmentsCaptor.getValue().size()); - ClientCompactionTaskGranularitySpec actual = granularitySpecArgumentCaptor.getValue(); - Assert.assertNotNull(actual); - ClientCompactionTaskGranularitySpec expected = new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null, null); - Assert.assertEquals(expected, actual); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = + new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null, null); + Assert.assertEquals(expectedGranularitySpec, taskPayload.getGranularitySpec()); } @Test public void testCompactWithMetricsSpecShouldSetPreserveExistingMetricsTrue() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1787,30 +1632,19 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor clientCompactionTaskQueryTuningConfigArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskQueryTuningConfig.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.anyInt(), - clientCompactionTaskQueryTuningConfigArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); - Assert.assertNotNull(clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue()); - Assert.assertNotNull(clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue().getAppendableIndexSpec()); - Assert.assertTrue(((OnheapIncrementalIndex.Spec) clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue().getAppendableIndexSpec()).isPreserveExistingMetrics()); + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + Assert.assertNotNull(taskPayload.getTuningConfig()); + Assert.assertNotNull(taskPayload.getTuningConfig().getAppendableIndexSpec()); + Assert.assertTrue(((OnheapIncrementalIndex.Spec) taskPayload.getTuningConfig() + .getAppendableIndexSpec()).isPreserveExistingMetrics()); } @Test public void testCompactWithoutMetricsSpecShouldSetPreserveExistingMetricsFalse() { - final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); - final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); + final OverlordClient mockClient = Mockito.mock(OverlordClient.class); + final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); + final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1850,23 +1684,11 @@ public class CompactSegmentsTest ) ); doCompactSegments(compactSegments, compactionConfigs); - ArgumentCaptor clientCompactionTaskQueryTuningConfigArgumentCaptor = ArgumentCaptor.forClass( - ClientCompactionTaskQueryTuningConfig.class); - Mockito.verify(mockIndexingServiceClient).compactSegments( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.anyInt(), - clientCompactionTaskQueryTuningConfigArgumentCaptor.capture(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); - Assert.assertNotNull(clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue()); - Assert.assertNotNull(clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue().getAppendableIndexSpec()); - Assert.assertFalse(((OnheapIncrementalIndex.Spec) clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue().getAppendableIndexSpec()).isPreserveExistingMetrics()); + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + Assert.assertNotNull(taskPayload.getTuningConfig()); + Assert.assertNotNull(taskPayload.getTuningConfig().getAppendableIndexSpec()); + Assert.assertFalse(((OnheapIncrementalIndex.Spec) taskPayload.getTuningConfig() + .getAppendableIndexSpec()).isPreserveExistingMetrics()); } private void verifySnapshot( @@ -2178,7 +2000,7 @@ public class CompactSegmentsTest return compactionConfigs; } - private class TestDruidLeaderClient extends DruidLeaderClient + private class TestOverlordClient extends NoopOverlordClient { private final ObjectMapper jsonMapper; @@ -2190,78 +2012,21 @@ public class CompactSegmentsTest private int compactVersionSuffix = 0; - private TestDruidLeaderClient(ObjectMapper jsonMapper) + private TestOverlordClient(ObjectMapper jsonMapper) { - super(null, new TestNodeDiscoveryProvider(), null, null); this.jsonMapper = jsonMapper; } @Override - public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException + public ListenableFuture findCurrentLeader() { - return new Request(httpMethod, new URL("http", "host", 8090, urlPath)); + throw new UnsupportedOperationException(); } @Override - public StringFullResponseHolder go(Request request) throws IOException + public ListenableFuture runTask(String taskId, Object taskObject) { - final String urlString = request.getUrl().toString(); - if (urlString.contains("/druid/indexer/v1/task")) { - return handleTask(request); - } else if (urlString.contains("/druid/indexer/v1/workers")) { - return handleWorkers(); - } else if (urlString.contains("/druid/indexer/v1/totalWorkerCapacity")) { - return handleTotalWorkerCapacity(); - } else if (urlString.contains("/druid/indexer/v1/waitingTasks") - || urlString.contains("/druid/indexer/v1/pendingTasks") - || urlString.contains("/druid/indexer/v1/runningTasks")) { - return createStringFullResponseHolder(jsonMapper.writeValueAsString(Collections.emptyList())); - } else if (urlString.contains(("/druid/indexer/v1/lockedIntervals"))) { - return handleLockedIntervals(); - } else { - throw new IAE("Cannot handle request for url[%s]", request.getUrl()); - } - } - - private StringFullResponseHolder createStringFullResponseHolder(String content) - { - final HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - final StringFullResponseHolder holder = new StringFullResponseHolder( - httpResponse, - StandardCharsets.UTF_8 - ); - holder.addChunk(content); - return holder; - } - - private StringFullResponseHolder handleWorkers() throws JsonProcessingException - { - final List workerInfos = new ArrayList<>(); - // There are 10 workers available in this test - for (int i = 0; i < 10; i++) { - workerInfos.add( - new IndexingWorkerInfo( - new IndexingWorker("http", "host", "8091", 1, "version"), - 0, - Collections.emptySet(), - Collections.emptyList(), - DateTimes.EPOCH, - null - ) - ); - } - return createStringFullResponseHolder(jsonMapper.writeValueAsString(workerInfos)); - } - - private StringFullResponseHolder handleTotalWorkerCapacity() throws JsonProcessingException - { - IndexingTotalWorkerCapacityInfo info = new IndexingTotalWorkerCapacityInfo(5, 10); - return createStringFullResponseHolder(jsonMapper.writeValueAsString(info)); - } - - private StringFullResponseHolder handleTask(Request request) throws IOException - { - final ClientTaskQuery taskQuery = jsonMapper.readValue(request.getContent().array(), ClientTaskQuery.class); + final ClientTaskQuery taskQuery = jsonMapper.convertValue(taskObject, ClientTaskQuery.class); if (!(taskQuery instanceof ClientCompactionTaskQuery)) { throw new IAE("Cannot run non-compaction task"); } @@ -2278,12 +2043,29 @@ public class CompactSegmentsTest .collect(Collectors.toList()); compactSegments(timeline, segments, compactionTaskQuery); - return createStringFullResponseHolder(jsonMapper.writeValueAsString(ImmutableMap.of("task", taskQuery.getId()))); + return Futures.immediateFuture(null); } - private StringFullResponseHolder handleLockedIntervals() throws IOException + @Override + public ListenableFuture>> findLockedIntervals(Map minTaskPriority) { - return createStringFullResponseHolder(jsonMapper.writeValueAsString(lockedIntervals)); + return Futures.immediateFuture(lockedIntervals); + } + + @Override + public ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ) + { + return Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator())); + } + + @Override + public ListenableFuture getTotalWorkerCapacity() + { + return Futures.immediateFuture(new IndexingTotalWorkerCapacityInfo(5, 10)); } private void compactSegments( @@ -2384,21 +2166,6 @@ public class CompactSegmentsTest } } - private static class TestNodeDiscoveryProvider extends DruidNodeDiscoveryProvider - { - @Override - public BooleanSupplier getForNode(DruidNode node, NodeRole nodeRole) - { - throw new UnsupportedOperationException(); - } - - @Override - public DruidNodeDiscovery getForNodeRole(NodeRole nodeRole) - { - return EasyMock.niceMock(DruidNodeDiscovery.class); - } - } - public static class StaticUtilsTest { @Test @@ -2465,4 +2232,18 @@ public class CompactSegmentsTest Assert.assertEquals(1, CompactSegments.findMaxNumTaskSlotsUsedByOneCompactionTask(tuningConfig)); } } + + private static ArgumentCaptor setUpMockClient(final OverlordClient mockClient) + { + final ArgumentCaptor payloadCaptor = ArgumentCaptor.forClass(Object.class); + Mockito.when(mockClient.taskStatuses(null, null, 0)) + .thenReturn(Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator()))); + Mockito.when(mockClient.findLockedIntervals(ArgumentMatchers.any())) + .thenReturn(Futures.immediateFuture(Collections.emptyMap())); + Mockito.when(mockClient.getTotalWorkerCapacity()) + .thenReturn(Futures.immediateFuture(new IndexingTotalWorkerCapacityInfo(0, 0))); + Mockito.when(mockClient.runTask(ArgumentMatchers.anyString(), payloadCaptor.capture())) + .thenReturn(Futures.immediateFuture(null)); + return payloadCaptor; + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java index 3bd25f4c197..8f43b36bbe5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java @@ -20,9 +20,9 @@ package org.apache.druid.server.coordinator.duty; import com.google.common.collect.ImmutableList; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -64,7 +64,7 @@ public class KillUnusedSegmentsTest @Mock private SegmentsMetadataManager segmentsMetadataManager; @Mock - private IndexingServiceClient indexingServiceClient; + private OverlordClient overlordClient; @Mock(answer = Answers.RETURNS_DEEP_STUBS) private DruidCoordinatorConfig config; @@ -131,7 +131,7 @@ public class KillUnusedSegmentsTest return unusedIntervals.size() <= limit ? unusedIntervals : unusedIntervals.subList(0, limit); }); - target = new KillUnusedSegments(segmentsMetadataManager, indexingServiceClient, config); + target = new KillUnusedSegments(segmentsMetadataManager, overlordClient, config); } @Test @@ -144,8 +144,8 @@ public class KillUnusedSegmentsTest ); target.run(params); - Mockito.verify(indexingServiceClient, Mockito.never()) - .killUnusedSegments(anyString(), anyString(), any(Interval.class)); + Mockito.verify(overlordClient, Mockito.never()) + .runKillTask(anyString(), anyString(), any(Interval.class)); } @Test @@ -153,12 +153,12 @@ public class KillUnusedSegmentsTest { Mockito.doReturn(Duration.standardDays(400)) .when(config).getCoordinatorKillDurationToRetain(); - target = new KillUnusedSegments(segmentsMetadataManager, indexingServiceClient, config); + target = new KillUnusedSegments(segmentsMetadataManager, overlordClient, config); // No unused segment is older than the retention period target.run(params); - Mockito.verify(indexingServiceClient, Mockito.never()) - .killUnusedSegments(anyString(), anyString(), any(Interval.class)); + Mockito.verify(overlordClient, Mockito.never()) + .runKillTask(anyString(), anyString(), any(Interval.class)); } @Test @@ -178,7 +178,7 @@ public class KillUnusedSegmentsTest // Duration to retain = -1 day, reinit target for config to take effect Mockito.doReturn(DURATION_TO_RETAIN.negated()) .when(config).getCoordinatorKillDurationToRetain(); - target = new KillUnusedSegments(segmentsMetadataManager, indexingServiceClient, config); + target = new KillUnusedSegments(segmentsMetadataManager, overlordClient, config); // Segments upto 1 day in the future are killed Interval expectedKillInterval = new Interval( @@ -193,7 +193,7 @@ public class KillUnusedSegmentsTest { Mockito.doReturn(true) .when(config).getCoordinatorKillIgnoreDurationToRetain(); - target = new KillUnusedSegments(segmentsMetadataManager, indexingServiceClient, config); + target = new KillUnusedSegments(segmentsMetadataManager, overlordClient, config); // All future and past unused segments are killed Interval expectedKillInterval = new Interval( @@ -208,7 +208,7 @@ public class KillUnusedSegmentsTest { Mockito.doReturn(1) .when(config).getCoordinatorKillMaxSegments(); - target = new KillUnusedSegments(segmentsMetadataManager, indexingServiceClient, config); + target = new KillUnusedSegments(segmentsMetadataManager, overlordClient, config); // Only 1 unused segment is killed runAndVerifyKillInterval(yearOldSegment.getInterval()); @@ -217,7 +217,7 @@ public class KillUnusedSegmentsTest private void runAndVerifyKillInterval(Interval expectedKillInterval) { target.run(params); - Mockito.verify(indexingServiceClient, Mockito.times(1)).killUnusedSegments( + Mockito.verify(overlordClient, Mockito.times(1)).runKillTask( ArgumentMatchers.anyString(), ArgumentMatchers.eq("DS1"), ArgumentMatchers.eq(expectedKillInterval) diff --git a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java index 4e59e913e23..5dec3154d6e 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.Futures; import it.unimi.dsi.fastutil.objects.Object2LongMap; import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.druid.client.CoordinatorServerView; @@ -33,13 +34,13 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.client.SegmentLoadInfo; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.metadata.UnknownSegmentIdsException; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.DruidCoordinator; @@ -591,27 +592,27 @@ public class DataSourcesResourceTest String interval = "2010-01-01_P1D"; Interval theInterval = Intervals.of(interval.replace('_', '/')); - IndexingServiceClient indexingServiceClient = EasyMock.createStrictMock(IndexingServiceClient.class); - indexingServiceClient.killUnusedSegments("api-issued", "datasource1", theInterval); - EasyMock.expectLastCall().once(); - EasyMock.replay(indexingServiceClient, server); + OverlordClient overlordClient = EasyMock.createStrictMock(OverlordClient.class); + EasyMock.expect(overlordClient.runKillTask("api-issued", "datasource1", theInterval)) + .andReturn(Futures.immediateFuture(null)); + EasyMock.replay(overlordClient, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null, null); + new DataSourcesResource(inventoryView, null, null, overlordClient, null, null); Response response = dataSourcesResource.killUnusedSegmentsInInterval("datasource1", interval); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(null, response.getEntity()); - EasyMock.verify(indexingServiceClient, server); + Assert.assertNull(response.getEntity()); + EasyMock.verify(overlordClient, server); } @Test public void testMarkAsUnusedAllSegmentsInDataSource() { - IndexingServiceClient indexingServiceClient = EasyMock.createStrictMock(IndexingServiceClient.class); - EasyMock.replay(indexingServiceClient, server); + OverlordClient overlordClient = EasyMock.createStrictMock(OverlordClient.class); + EasyMock.replay(overlordClient, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null, null); + new DataSourcesResource(inventoryView, null, null, overlordClient, null, null); try { Response response = dataSourcesResource.markAsUnusedAllSegmentsOrKillUnusedSegmentsInInterval("datasource", "true", "???"); @@ -624,7 +625,7 @@ public class DataSourcesResourceTest // expected } - EasyMock.verify(indexingServiceClient, server); + EasyMock.verify(overlordClient, server); } @Test diff --git a/server/src/test/java/org/apache/druid/server/http/OverlordProxyServletTest.java b/server/src/test/java/org/apache/druid/server/http/OverlordProxyServletTest.java index 7e99d827bed..cb9f5a51253 100644 --- a/server/src/test/java/org/apache/druid/server/http/OverlordProxyServletTest.java +++ b/server/src/test/java/org/apache/druid/server/http/OverlordProxyServletTest.java @@ -19,21 +19,24 @@ package org.apache.druid.server.http; -import org.apache.druid.discovery.DruidLeaderClient; +import com.google.common.util.concurrent.Futures; +import org.apache.druid.rpc.indexing.OverlordClient; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; import javax.servlet.http.HttpServletRequest; import java.net.URI; +import java.net.URISyntaxException; public class OverlordProxyServletTest { @Test - public void testRewriteURI() + public void testRewriteURI() throws URISyntaxException { - DruidLeaderClient druidLeaderClient = EasyMock.createMock(DruidLeaderClient.class); - EasyMock.expect(druidLeaderClient.findCurrentLeader()).andReturn("https://overlord:port"); + OverlordClient overlordClient = EasyMock.createMock(OverlordClient.class); + EasyMock.expect(overlordClient.findCurrentLeader()) + .andReturn(Futures.immediateFuture(new URI("https://overlord:port"))); HttpServletRequest request = EasyMock.createMock(HttpServletRequest.class); EasyMock.expect(request.getQueryString()).andReturn("param1=test¶m2=test2").anyTimes(); @@ -41,10 +44,11 @@ public class OverlordProxyServletTest // %3A is a colon; test to make sure urlencoded paths work right. EasyMock.expect(request.getRequestURI()).andReturn("/druid/over%3Alord/worker").anyTimes(); - EasyMock.replay(druidLeaderClient, request); + EasyMock.replay(overlordClient, request); - URI uri = URI.create(new OverlordProxyServlet(druidLeaderClient, null, null).rewriteTarget(request)); + URI uri = URI.create(new OverlordProxyServlet(overlordClient, null, null).rewriteTarget(request)); Assert.assertEquals("https://overlord:port/druid/over%3Alord/worker?param1=test¶m2=test2", uri.toString()); - } + EasyMock.verify(overlordClient, request); + } } diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 73adc79a4f3..6d9be301d18 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -41,8 +41,6 @@ import org.apache.druid.client.CoordinatorSegmentWatcherConfig; import org.apache.druid.client.CoordinatorServerView; import org.apache.druid.client.HttpServerInventoryViewResource; import org.apache.druid.client.coordinator.Coordinator; -import org.apache.druid.client.indexing.HttpIndexingServiceClient; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.ConditionalMultibind; import org.apache.druid.guice.ConfigProvider; @@ -221,8 +219,6 @@ public class CliCoordinator extends ServerRunnable .toProvider(AuditManagerProvider.class) .in(ManageLifecycle.class); - binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); - binder.bind(LookupCoordinatorManager.class).in(LazySingleton.class); binder.bind(CoordinatorServerView.class); binder.bind(DruidCoordinator.class); 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 4c365c63a90..1bbe1a0e9c8 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -39,7 +39,6 @@ import com.google.inject.servlet.GuiceFilter; import com.google.inject.util.Providers; import org.apache.druid.audit.AuditManager; import org.apache.druid.client.indexing.IndexingService; -import org.apache.druid.client.indexing.IndexingServiceSelectorConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; @@ -150,7 +149,8 @@ import java.util.Set; ) public class CliOverlord extends ServerRunnable { - private static Logger log = new Logger(CliOverlord.class); + private static final Logger log = new Logger(CliOverlord.class); + private static final String DEFAULT_SERVICE_NAME = "druid/overlord"; protected static final List UNSECURED_PATHS = ImmutableList.of( "/druid/indexer/v1/isLeader", @@ -185,7 +185,7 @@ public class CliOverlord extends ServerRunnable if (standalone) { binder.bindConstant() .annotatedWith(Names.named("serviceName")) - .to(IndexingServiceSelectorConfig.DEFAULT_SERVICE_NAME); + .to(DEFAULT_SERVICE_NAME); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8090); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8290); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 2c60e9eda1c..db4a10af81c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -50,8 +50,8 @@ import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.JsonParserIterator; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.coordinator.Coordinator; -import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidLeaderClient; @@ -66,6 +66,7 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.InputStreamFullResponseHandler; import org.apache.druid.java.util.http.client.response.InputStreamFullResponseHolder; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.server.DruidNode; @@ -86,6 +87,7 @@ import org.jboss.netty.handler.codec.http.HttpMethod; import javax.annotation.Nullable; import javax.servlet.http.HttpServletResponse; +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -215,7 +217,7 @@ public class SystemSchema extends AbstractSchema final FilteredServerInventoryView serverInventoryView, final AuthorizerMapper authorizerMapper, final @Coordinator DruidLeaderClient coordinatorDruidLeaderClient, - final @IndexingService DruidLeaderClient overlordDruidLeaderClient, + final OverlordClient overlordClient, final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, final ObjectMapper jsonMapper ) @@ -223,10 +225,10 @@ public class SystemSchema extends AbstractSchema Preconditions.checkNotNull(serverView, "serverView"); this.tableMap = ImmutableMap.of( SEGMENTS_TABLE, new SegmentsTable(druidSchema, metadataView, jsonMapper, authorizerMapper), - SERVERS_TABLE, new ServersTable(druidNodeDiscoveryProvider, serverInventoryView, authorizerMapper, overlordDruidLeaderClient, coordinatorDruidLeaderClient), + SERVERS_TABLE, new ServersTable(druidNodeDiscoveryProvider, serverInventoryView, authorizerMapper, overlordClient, coordinatorDruidLeaderClient), SERVER_SEGMENTS_TABLE, new ServerSegmentsTable(serverView, authorizerMapper), - TASKS_TABLE, new TasksTable(overlordDruidLeaderClient, jsonMapper, authorizerMapper), - SUPERVISOR_TABLE, new SupervisorsTable(overlordDruidLeaderClient, jsonMapper, authorizerMapper) + TASKS_TABLE, new TasksTable(overlordClient, authorizerMapper), + SUPERVISOR_TABLE, new SupervisorsTable(overlordClient, authorizerMapper) ); } @@ -496,21 +498,21 @@ public class SystemSchema extends AbstractSchema private final AuthorizerMapper authorizerMapper; private final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider; private final FilteredServerInventoryView serverInventoryView; - private final DruidLeaderClient overlordLeaderClient; + private final OverlordClient overlordClient; private final DruidLeaderClient coordinatorLeaderClient; public ServersTable( DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, FilteredServerInventoryView serverInventoryView, AuthorizerMapper authorizerMapper, - DruidLeaderClient overlordLeaderClient, + OverlordClient overlordClient, DruidLeaderClient coordinatorLeaderClient ) { this.authorizerMapper = authorizerMapper; this.druidNodeDiscoveryProvider = druidNodeDiscoveryProvider; this.serverInventoryView = serverInventoryView; - this.overlordLeaderClient = overlordLeaderClient; + this.overlordClient = overlordClient; this.coordinatorLeaderClient = coordinatorLeaderClient; } @@ -538,13 +540,21 @@ public class SystemSchema extends AbstractSchema String tmpCoordinatorLeader = ""; String tmpOverlordLeader = ""; + try { tmpCoordinatorLeader = coordinatorLeaderClient.findCurrentLeader(); - tmpOverlordLeader = overlordLeaderClient.findCurrentLeader(); } - catch (ISE ignored) { + catch (Exception ignored) { // no reason to kill the results if something is sad and there are no leaders } + + try { + tmpOverlordLeader = FutureUtils.getUnchecked(overlordClient.findCurrentLeader(), true).toString(); + } + catch (Exception ignored) { + // no reason to kill the results if something is sad and there are no leaders + } + final String coordinatorLeader = tmpCoordinatorLeader; final String overlordLeader = tmpOverlordLeader; @@ -761,18 +771,15 @@ public class SystemSchema extends AbstractSchema */ static class TasksTable extends AbstractTable implements ScannableTable { - private final DruidLeaderClient druidLeaderClient; - private final ObjectMapper jsonMapper; + private final OverlordClient overlordClient; private final AuthorizerMapper authorizerMapper; public TasksTable( - DruidLeaderClient druidLeaderClient, - ObjectMapper jsonMapper, + OverlordClient overlordClient, AuthorizerMapper authorizerMapper ) { - this.druidLeaderClient = druidLeaderClient; - this.jsonMapper = jsonMapper; + this.overlordClient = overlordClient; this.authorizerMapper = authorizerMapper; } @@ -795,7 +802,7 @@ public class SystemSchema extends AbstractSchema { private final CloseableIterator it; - public TasksEnumerable(JsonParserIterator tasks) + public TasksEnumerable(CloseableIterator tasks) { this.it = getAuthorizedTasks(tasks, root); } @@ -874,11 +881,11 @@ public class SystemSchema extends AbstractSchema } } - return new TasksEnumerable(getTasks(druidLeaderClient, jsonMapper)); + return new TasksEnumerable(FutureUtils.getUnchecked(overlordClient.taskStatuses(null, null, null), true)); } private CloseableIterator getAuthorizedTasks( - JsonParserIterator it, + CloseableIterator it, DataContext root ) { @@ -902,39 +909,20 @@ public class SystemSchema extends AbstractSchema } - //Note that overlord must be up to get tasks - private static JsonParserIterator getTasks( - DruidLeaderClient indexingServiceClient, - ObjectMapper jsonMapper - ) - { - return getThingsFromLeaderNode( - "/druid/indexer/v1/tasks", - new TypeReference() - { - }, - indexingServiceClient, - jsonMapper - ); - } - /** * This table contains a row per supervisor task. */ static class SupervisorsTable extends AbstractTable implements ScannableTable { - private final DruidLeaderClient druidLeaderClient; - private final ObjectMapper jsonMapper; + private final OverlordClient overlordClient; private final AuthorizerMapper authorizerMapper; public SupervisorsTable( - DruidLeaderClient druidLeaderClient, - ObjectMapper jsonMapper, + OverlordClient overlordClient, AuthorizerMapper authorizerMapper ) { - this.druidLeaderClient = druidLeaderClient; - this.jsonMapper = jsonMapper; + this.overlordClient = overlordClient; this.authorizerMapper = authorizerMapper; } @@ -958,7 +946,7 @@ public class SystemSchema extends AbstractSchema { private final CloseableIterator it; - public SupervisorsEnumerable(JsonParserIterator tasks) + public SupervisorsEnumerable(CloseableIterator tasks) { this.it = getAuthorizedSupervisors(tasks, root); } @@ -1016,11 +1004,11 @@ public class SystemSchema extends AbstractSchema } } - return new SupervisorsEnumerable(getSupervisors(druidLeaderClient, jsonMapper)); + return new SupervisorsEnumerable(FutureUtils.getUnchecked(overlordClient.supervisorStatuses(), true)); } private CloseableIterator getAuthorizedSupervisors( - JsonParserIterator it, + CloseableIterator it, DataContext root ) { @@ -1043,23 +1031,6 @@ public class SystemSchema extends AbstractSchema } } - // Note that overlord must be up to get supervisor tasks, otherwise queries to sys.supervisors table - // will fail with internal server error (HTTP 500) - private static JsonParserIterator getSupervisors( - DruidLeaderClient indexingServiceClient, - ObjectMapper jsonMapper - ) - { - return getThingsFromLeaderNode( - "/druid/indexer/v1/supervisor?system", - new TypeReference() - { - }, - indexingServiceClient, - jsonMapper - ); - } - public static JsonParserIterator getThingsFromLeaderNode( String query, TypeReference typeRef, @@ -1104,7 +1075,7 @@ public class SystemSchema extends AbstractSchema ); } - private static CloseableIterator wrap(Iterator iterator, JsonParserIterator it) + private static CloseableIterator wrap(Iterator iterator, Closeable closer) { return new CloseableIterator() { @@ -1114,7 +1085,7 @@ public class SystemSchema extends AbstractSchema final boolean hasNext = iterator.hasNext(); if (!hasNext) { try { - it.close(); + closer.close(); } catch (IOException e) { throw new RuntimeException(e); @@ -1132,7 +1103,7 @@ public class SystemSchema extends AbstractSchema @Override public void close() throws IOException { - it.close(); + closer.close(); } }; } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java index 6b3bb73feba..1d646214ec7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java @@ -32,6 +32,7 @@ import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.coordinator.Coordinator; import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.guice.LazySingleton; @@ -40,6 +41,7 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.query.lookup.LookupReferencesManager; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.server.QueryLifecycleFactory; @@ -129,6 +131,7 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupReferencesManager); binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); binder.bind(ServiceEmitter.class).toInstance(new ServiceEmitter("", "", null)); + binder.bind(OverlordClient.class).to(NoopOverlordClient.class); }, new LifecycleModule(), target); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 511a76b18ad..82eae68c332 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -19,11 +19,13 @@ package org.apache.druid.sql.calcite.schema; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.Futures; import junitparams.converters.Nullable; import org.apache.calcite.DataContext; import org.apache.calcite.adapter.java.JavaTypeFactory; @@ -48,7 +50,10 @@ import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.NodeRole; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; +import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -57,7 +62,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.Closer; 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.InputStreamFullResponseHandler; import org.apache.druid.java.util.http.client.response.InputStreamFullResponseHolder; import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; import org.apache.druid.query.QueryRunnerFactoryConglomerate; @@ -65,6 +69,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.TestHelper; @@ -102,11 +107,7 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentStatusInCluster; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; -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.joda.time.DateTime; import org.junit.AfterClass; import org.junit.Assert; @@ -118,7 +119,8 @@ import org.junit.rules.TemporaryFolder; import java.io.File; import java.io.IOException; -import java.net.URL; +import java.net.URI; +import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; @@ -153,7 +155,7 @@ public class SystemSchemaTest extends CalciteTestBase private SpecificSegmentsQuerySegmentWalker walker; private DruidLeaderClient client; private DruidLeaderClient coordinatorClient; - private DruidLeaderClient overlordClient; + private OverlordClient overlordClient; private TimelineServerView serverView; private ObjectMapper mapper; private StringFullResponseHolder responseHolder; @@ -189,7 +191,7 @@ public class SystemSchemaTest extends CalciteTestBase serverView = EasyMock.createNiceMock(TimelineServerView.class); client = EasyMock.createMock(DruidLeaderClient.class); coordinatorClient = EasyMock.createMock(DruidLeaderClient.class); - overlordClient = EasyMock.createMock(DruidLeaderClient.class); + overlordClient = EasyMock.createMock(OverlordClient.class); mapper = TestHelper.makeJsonMapper(); responseHolder = EasyMock.createMock(StringFullResponseHolder.class); responseHandler = EasyMock.createMockBuilder(BytesAccumulatingResponseHandler.class) @@ -272,7 +274,7 @@ public class SystemSchemaTest extends CalciteTestBase serverInventoryView, EasyMock.createStrictMock(AuthorizerMapper.class), client, - client, + overlordClient, druidNodeDiscoveryProvider, mapper ); @@ -753,7 +755,7 @@ public class SystemSchemaTest extends CalciteTestBase } @Test - public void testServersTable() + public void testServersTable() throws URISyntaxException { SystemSchema.ServersTable serversTable = EasyMock.createMockBuilder(SystemSchema.ServersTable.class) @@ -809,7 +811,8 @@ public class SystemSchemaTest extends CalciteTestBase EasyMock.expect(indexerNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(indexer)).once(); EasyMock.expect(coordinatorClient.findCurrentLeader()).andReturn(coordinator.getDruidNode().getHostAndPortToUse()).once(); - EasyMock.expect(overlordClient.findCurrentLeader()).andReturn(overlord.getDruidNode().getHostAndPortToUse()).once(); + EasyMock.expect(overlordClient.findCurrentLeader()) + .andReturn(Futures.immediateFuture(new URI(overlord.getDruidNode().getHostAndPortToUse()))).once(); final List servers = new ArrayList<>(); servers.add(mockDataServer(historical1.getDruidNode().getHostAndPortToUse(), 200L, 1000L, "tier")); @@ -1152,18 +1155,10 @@ public class SystemSchemaTest extends CalciteTestBase { SystemSchema.TasksTable tasksTable = EasyMock.createMockBuilder(SystemSchema.TasksTable.class) - .withConstructor(client, mapper, authMapper) + .withConstructor(overlordClient, authMapper) .createMock(); EasyMock.replay(tasksTable); - EasyMock.expect(client.makeRequest(HttpMethod.GET, "/druid/indexer/v1/tasks")).andReturn(request).anyTimes(); - - - HttpResponse httpResp = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - InputStreamFullResponseHolder responseHolder = new InputStreamFullResponseHolder(httpResp); - - EasyMock.expect(client.go(EasyMock.eq(request), EasyMock.anyObject(InputStreamFullResponseHandler.class))).andReturn(responseHolder).once(); - EasyMock.expect(request.getUrl()).andReturn(new URL("http://test-host:1234/druid/indexer/v1/tasks")).anyTimes(); String json = "[{\n" + "\t\"id\": \"index_wikipedia_2018-09-20T22:33:44.911Z\",\n" @@ -1198,11 +1193,16 @@ public class SystemSchemaTest extends CalciteTestBase + "\t\"dataSource\": \"wikipedia\",\n" + "\t\"errorMsg\": null\n" + "}]"; - byte[] bytesToWrite = json.getBytes(StandardCharsets.UTF_8); - responseHolder.addChunk(bytesToWrite); - responseHolder.done(); - EasyMock.replay(client, request, responseHandler); + EasyMock.expect(overlordClient.taskStatuses(null, null, null)).andReturn( + Futures.immediateFuture( + CloseableIterators.withEmptyBaggage( + mapper.readValue(json, new TypeReference>() {}).iterator() + ) + ) + ); + + EasyMock.replay(overlordClient, request, responseHandler); DataContext dataContext = createDataContext(Users.SUPER); final List rows = tasksTable.scan(dataContext).toList(); @@ -1243,13 +1243,9 @@ public class SystemSchemaTest extends CalciteTestBase } @Test - public void testTasksTableAuth() throws Exception + public void testTasksTableAuth() { - SystemSchema.TasksTable tasksTable = new SystemSchema.TasksTable(client, mapper, authMapper); - - EasyMock.expect(client.makeRequest(HttpMethod.GET, "/druid/indexer/v1/tasks")) - .andReturn(request) - .anyTimes(); + SystemSchema.TasksTable tasksTable = new SystemSchema.TasksTable(overlordClient, authMapper); String json = "[{\n" + "\t\"id\": \"index_wikipedia_2018-09-20T22:33:44.911Z\",\n" @@ -1285,18 +1281,15 @@ public class SystemSchemaTest extends CalciteTestBase + "\t\"errorMsg\": null\n" + "}]"; - HttpResponse httpResp = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + EasyMock.expect(overlordClient.taskStatuses(null, null, null)).andAnswer( + () -> Futures.immediateFuture( + CloseableIterators.withEmptyBaggage( + mapper.readValue(json, new TypeReference>() {}).iterator() + ) + ) + ).anyTimes(); - EasyMock.expect(client.go(EasyMock.eq(request), EasyMock.anyObject())) - .andReturn(createFullResponseHolder(httpResp, json)) - .andReturn(createFullResponseHolder(httpResp, json)) - .andReturn(createFullResponseHolder(httpResp, json)); - - EasyMock.expect(request.getUrl()) - .andReturn(new URL("http://test-host:1234/druid/indexer/v1/tasks")) - .anyTimes(); - - EasyMock.replay(client, request, responseHandler); + EasyMock.replay(overlordClient); // Verify that no row is returned for Datasource Write user List rows = tasksTable @@ -1320,28 +1313,11 @@ public class SystemSchemaTest extends CalciteTestBase @Test public void testSupervisorTable() throws Exception { - - SystemSchema.SupervisorsTable supervisorTable = EasyMock.createMockBuilder(SystemSchema.SupervisorsTable.class) - .withConstructor( - client, - mapper, - authMapper - ) - .createMock(); + SystemSchema.SupervisorsTable supervisorTable = + EasyMock.createMockBuilder(SystemSchema.SupervisorsTable.class) + .withConstructor(overlordClient, authMapper) + .createMock(); EasyMock.replay(supervisorTable); - EasyMock.expect(client.makeRequest(HttpMethod.GET, "/druid/indexer/v1/supervisor?system")) - .andReturn(request) - .anyTimes(); - - HttpResponse httpResp = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - InputStreamFullResponseHolder responseHolder = new InputStreamFullResponseHolder(httpResp); - - EasyMock.expect(client.go(EasyMock.eq(request), EasyMock.anyObject(InputStreamFullResponseHandler.class))).andReturn(responseHolder).once(); - - EasyMock.expect(responseHandler.getStatus()).andReturn(httpResp.getStatus().getCode()).anyTimes(); - EasyMock.expect(request.getUrl()) - .andReturn(new URL("http://test-host:1234/druid/indexer/v1/supervisor?system")) - .anyTimes(); String json = "[{\n" + "\t\"id\": \"wikipedia\",\n" @@ -1355,11 +1331,15 @@ public class SystemSchemaTest extends CalciteTestBase + "\t\"suspended\": false\n" + "}]"; - byte[] bytesToWrite = json.getBytes(StandardCharsets.UTF_8); - responseHolder.addChunk(bytesToWrite); - responseHolder.done(); + EasyMock.expect(overlordClient.supervisorStatuses()).andReturn( + Futures.immediateFuture( + CloseableIterators.withEmptyBaggage( + mapper.readValue(json, new TypeReference>() {}).iterator() + ) + ) + ); - EasyMock.replay(client, request, responseHandler); + EasyMock.replay(overlordClient); DataContext dataContext = createDataContext(Users.SUPER); final List rows = supervisorTable.scan(dataContext).toList(); @@ -1381,14 +1361,10 @@ public class SystemSchemaTest extends CalciteTestBase } @Test - public void testSupervisorTableAuth() throws Exception + public void testSupervisorTableAuth() { SystemSchema.SupervisorsTable supervisorTable = - new SystemSchema.SupervisorsTable(client, mapper, createAuthMapper()); - - EasyMock.expect(client.makeRequest(HttpMethod.GET, "/druid/indexer/v1/supervisor?system")) - .andReturn(request) - .anyTimes(); + new SystemSchema.SupervisorsTable(overlordClient, createAuthMapper()); final String json = "[{\n" + "\t\"id\": \"wikipedia\",\n" @@ -1402,20 +1378,15 @@ public class SystemSchemaTest extends CalciteTestBase + "\t\"suspended\": false\n" + "}]"; - HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - EasyMock.expect(client.go(EasyMock.eq(request), EasyMock.anyObject())) - .andReturn(createFullResponseHolder(httpResponse, json)) - .andReturn(createFullResponseHolder(httpResponse, json)) - .andReturn(createFullResponseHolder(httpResponse, json)); + EasyMock.expect(overlordClient.supervisorStatuses()).andAnswer( + () -> Futures.immediateFuture( + CloseableIterators.withEmptyBaggage( + mapper.readValue(json, new TypeReference>() {}).iterator() + ) + ) + ).anyTimes(); - EasyMock.expect(responseHandler.getStatus()) - .andReturn(httpResponse.getStatus().getCode()) - .anyTimes(); - EasyMock.expect(request.getUrl()) - .andReturn(new URL("http://test-host:1234/druid/indexer/v1/supervisor?system")) - .anyTimes(); - - EasyMock.replay(client, request, responseHandler); + EasyMock.replay(overlordClient); // Verify that no row is returned for Datasource Write user List rows = supervisorTable diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 905c28c5fad..bab3e0957e8 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Injector; import com.google.inject.Key; @@ -31,6 +32,7 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.ServerInventoryView; import org.apache.druid.client.ServerView; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.discovery.DruidNodeDiscovery; @@ -44,6 +46,7 @@ import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.DruidNode; @@ -77,6 +80,8 @@ import org.joda.time.Duration; import javax.annotation.Nullable; import java.io.File; +import java.net.URI; +import java.net.URISyntaxException; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -330,7 +335,6 @@ public class CalciteTests final AuthorizerMapper authorizerMapper ) { - final DruidNode coordinatorNode = new DruidNode("test-coordinator", "dummy", false, 8081, null, true, false); FakeDruidNodeDiscoveryProvider provider = new FakeDruidNodeDiscoveryProvider( ImmutableMap.of( @@ -339,11 +343,6 @@ public class CalciteTests ); final DruidNode overlordNode = new DruidNode("test-overlord", "dummy", false, 8090, null, true, false); - FakeDruidNodeDiscoveryProvider overlordProvider = new FakeDruidNodeDiscoveryProvider( - ImmutableMap.of( - NodeRole.OVERLORD, new FakeDruidNodeDiscovery(ImmutableMap.of(NodeRole.OVERLORD, coordinatorNode)) - ) - ); final DruidLeaderClient druidLeaderClient = new DruidLeaderClient( new FakeHttpClient(), @@ -358,16 +357,16 @@ public class CalciteTests } }; - final DruidLeaderClient overlordLeaderClient = new DruidLeaderClient( - new FakeHttpClient(), - overlordProvider, - NodeRole.OVERLORD, - "/simple/leader" - ) { + final OverlordClient overlordClient = new NoopOverlordClient() { @Override - public String findCurrentLeader() + public ListenableFuture findCurrentLeader() { - return overlordNode.getHostAndPortToUse(); + try { + return Futures.immediateFuture(new URI(overlordNode.getHostAndPortToUse())); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } } }; @@ -383,7 +382,7 @@ public class CalciteTests new FakeServerInventoryView(), authorizerMapper, druidLeaderClient, - overlordLeaderClient, + overlordClient, provider, getJsonMapper() ); diff --git a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java index 13a52264836..76abef00542 100644 --- a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java @@ -32,6 +32,7 @@ import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.coordinator.Coordinator; import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.guice.DruidGuiceExtensions; @@ -49,6 +50,7 @@ import org.apache.druid.query.GenericQueryMetricsFactory; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.server.QueryScheduler; @@ -206,6 +208,7 @@ public class SqlModuleTest .in(LazySingleton.class); binder.bind(ResponseContextConfig.class).toInstance(SqlResourceTest.TEST_RESPONSE_CONTEXT_CONFIG); binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); + binder.bind(OverlordClient.class).to(NoopOverlordClient.class); }, sqlModule, new TestViewManagerModule()