mirror of https://github.com/apache/druid.git
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.
This commit is contained in:
parent
f742bb7376
commit
2f9619a96f
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -184,10 +184,7 @@ public class HeapMemoryTaskStorage implements TaskStorage
|
|||
return listBuilder.build();
|
||||
}
|
||||
|
||||
public List<TaskInfo<Task, TaskStatus>> getRecentlyCreatedAlreadyFinishedTaskInfo(
|
||||
CompleteTaskLookup taskLookup,
|
||||
@Nullable String datasource
|
||||
)
|
||||
public List<TaskInfo<Task, TaskStatus>> getRecentlyCreatedAlreadyFinishedTaskInfo(CompleteTaskLookup taskLookup)
|
||||
{
|
||||
final Ordering<TaskStuff> createdDateDesc = new Ordering<TaskStuff>()
|
||||
{
|
||||
|
@ -216,17 +213,15 @@ public class HeapMemoryTaskStorage implements TaskStorage
|
|||
)
|
||||
{
|
||||
final List<TaskInfo<Task, TaskStatus>> 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<TaskLookupType, TaskLookup> 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<String> 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<Task, TaskStatus> 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()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<TaskLookupType, TaskLookup> theTaskLookups = processTaskLookups(taskLookups);
|
||||
Map<TaskLookupType, TaskLookup> 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<TaskLookupType, TaskLookup> processedTaskLookups = processTaskLookups(taskLookups);
|
||||
Map<TaskLookupType, TaskLookup> 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<TaskLookupType, TaskLookup> processTaskLookups(
|
||||
Map<TaskLookupType, TaskLookup> taskLookups
|
||||
)
|
||||
{
|
||||
Map<TaskLookupType, TaskLookup> processedTaskLookups = Maps.newHashMapWithExpectedSize(taskLookups.size());
|
||||
for (Entry<TaskLookupType, TaskLookup> 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)
|
||||
{
|
||||
|
|
|
@ -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<TaskLookup.TaskLookupType, TaskLookup> processTaskLookups(
|
||||
final Map<TaskLookup.TaskLookupType, TaskLookup> taskLookups,
|
||||
final DateTime minCreationTimestamp
|
||||
)
|
||||
{
|
||||
final Map<TaskLookup.TaskLookupType, TaskLookup> retVal = new LinkedHashMap<>();
|
||||
|
||||
for (Map.Entry<TaskLookup.TaskLookupType, TaskLookup> 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;
|
||||
}
|
||||
}
|
|
@ -138,7 +138,7 @@ public class OverlordResource
|
|||
private final AuthConfig authConfig;
|
||||
|
||||
private AtomicReference<WorkerBehaviorConfig> workerConfigRef = null;
|
||||
private static final List API_TASK_STATES = ImmutableList.of("pending", "waiting", "running", "complete");
|
||||
private static final List<String> 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.
|
||||
*/
|
||||
|
|
|
@ -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<String> taskIds = ImmutableSet.copyOf(completedTasks.keySet());
|
||||
Map<String, TaskStatus> 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<Map<String, TaskStatus>>()
|
||||
{
|
||||
}
|
||||
);
|
||||
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<String> taskIds = ImmutableSet.copyOf(completedTasks.keySet());
|
||||
Either<Throwable, Map<String, TaskStatus>> 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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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<TaskInfo<Task, TaskStatus>> 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<TaskInfo<Task, TaskStatus>> 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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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")
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -96,7 +96,7 @@ public class SeekableStreamIndexTaskClientAsyncImplTest
|
|||
{
|
||||
final Map<Integer, Map<Integer, Long>> 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<Integer, Long> 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<Integer, Long> 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<Integer, Long> 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<Integer, Long> 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<Integer, Long> 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<Integer, Long> 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<Integer, Long> 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<String, Object> 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(),
|
||||
|
|
|
@ -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<String, TaskAnnouncement> 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<String, TaskAnnouncement> 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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<String> 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
|
||||
{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<T> implements Iterator<T>, Closeable
|
||||
public class JsonParserIterator<T> implements CloseableIterator<T>
|
||||
{
|
||||
private static final Logger LOG = new Logger(JsonParserIterator.class);
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<DataSegment> segments,
|
||||
int compactionTaskPriority,
|
||||
@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig,
|
||||
@Nullable ClientCompactionTaskGranularitySpec granularitySpec,
|
||||
@Nullable ClientCompactionTaskDimensionsSpec dimensionsSpec,
|
||||
@Nullable AggregatorFactory[] metricsSpec,
|
||||
@Nullable ClientCompactionTaskTransformSpec transformSpec,
|
||||
@Nullable Boolean dropExisting,
|
||||
@Nullable Map<String, Object> 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<String, Object> 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<String, Object> 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<IndexingWorkerInfo> workers = jsonMapper.readValue(
|
||||
response.getContent(),
|
||||
new TypeReference<Collection<IndexingWorkerInfo>>() {}
|
||||
);
|
||||
|
||||
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<IndexingTotalWorkerCapacityInfo>() {}
|
||||
);
|
||||
return indexingTotalWorkerCapacityInfo.getMaximumCapacityWithAutoScale();
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TaskStatusPlus> 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<TaskStatusPlus> tasks = new ArrayList<>();
|
||||
final Set<String> taskIdsSeen = new HashSet<>();
|
||||
|
||||
final Iterable<TaskStatusPlus> 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<TaskStatusPlus> 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<List<TaskStatusPlus>>()
|
||||
{
|
||||
}
|
||||
);
|
||||
}
|
||||
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<TaskStatusResponse>()
|
||||
{
|
||||
}
|
||||
);
|
||||
}
|
||||
catch (IOException | InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public TaskStatusPlus getLastCompleteTask()
|
||||
{
|
||||
final List<TaskStatusPlus> 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<TaskPayloadResponse>()
|
||||
{
|
||||
}
|
||||
);
|
||||
}
|
||||
catch (IOException | InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Map<String, Object> 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<String, List<Interval>> getLockedIntervals(Map<String, Integer> minTaskPriority)
|
||||
{
|
||||
try {
|
||||
final StringFullResponseHolder responseHolder = druidLeaderClient.go(
|
||||
druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/lockedIntervals")
|
||||
.setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(minTaskPriority))
|
||||
);
|
||||
|
||||
final Map<String, List<Interval>> response = jsonMapper.readValue(
|
||||
responseHolder.getContent(),
|
||||
new TypeReference<Map<String, List<Interval>>>()
|
||||
{
|
||||
}
|
||||
);
|
||||
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<String, Object> 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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<DataSegment> segments,
|
||||
int compactionTaskPriority,
|
||||
@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig,
|
||||
@Nullable ClientCompactionTaskGranularitySpec granularitySpec,
|
||||
@Nullable ClientCompactionTaskDimensionsSpec dimensionsSpec,
|
||||
@Nullable AggregatorFactory[] metricsSpec,
|
||||
@Nullable ClientCompactionTaskTransformSpec transformSpec,
|
||||
@Nullable Boolean dropExisting,
|
||||
@Nullable Map<String, Object> 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<TaskStatusPlus> getActiveTasks();
|
||||
|
||||
TaskStatusResponse getTaskStatus(String taskId);
|
||||
|
||||
@Nullable
|
||||
TaskStatusPlus getLastCompleteTask();
|
||||
|
||||
@Nullable
|
||||
TaskPayloadResponse getTaskPayload(String taskId);
|
||||
|
||||
@Nullable
|
||||
Map<String, Object> 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<String, List<Interval>> getLockedIntervals(Map<String, Integer> minTaskPriority);
|
||||
|
||||
SamplerResponse sample(SamplerSpec samplerSpec);
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String> 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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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(),
|
||||
|
|
|
@ -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<URI> 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<Void> 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<String> 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<Void> 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<CloseableIterator<TaskStatusPlus>> 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<Map<String, TaskStatus>> taskStatuses(Set<String> taskIds);
|
||||
|
||||
/**
|
||||
* Returns {@link TaskStatusResponse} for a particular task ID. This includes somewhat more information than
|
||||
* the {@link TaskStatus} returned by {@link #taskStatuses(Set)}.
|
||||
*/
|
||||
ListenableFuture<TaskStatusResponse> 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<Map<String, Object>> 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<TaskPayloadResponse> taskPayload(String taskId);
|
||||
|
||||
/**
|
||||
* Returns all current supervisor statuses.
|
||||
*/
|
||||
ListenableFuture<CloseableIterator<SupervisorStatus>> 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<Map<String, List<Interval>>> findLockedIntervals(Map<String, Integer> 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<Integer> killPendingSegments(String dataSource, Interval interval);
|
||||
|
||||
/**
|
||||
* Returns information about workers.
|
||||
*/
|
||||
ListenableFuture<List<IndexingWorkerInfo>> getWorkers();
|
||||
|
||||
/**
|
||||
* Returns total worker capacity details.
|
||||
*/
|
||||
ListenableFuture<IndexingTotalWorkerCapacityInfo> getTotalWorkerCapacity();
|
||||
|
||||
/**
|
||||
* Returns a copy of this client with a different retry policy.
|
||||
*/
|
||||
OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy);
|
||||
}
|
||||
|
|
|
@ -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<URI> 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<Void> runTask(final String taskId, final Object taskObject)
|
||||
{
|
||||
|
@ -65,7 +99,8 @@ public class OverlordClientImpl implements OverlordClient
|
|||
new BytesFullResponseHandler()
|
||||
),
|
||||
holder -> {
|
||||
final Map<String, Object> map = deserialize(holder, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
|
||||
final Map<String, Object> 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<CloseableIterator<TaskStatusPlus>> 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<Map<String, TaskStatus>> taskStatuses(final Set<String> taskIds)
|
||||
{
|
||||
|
@ -100,7 +168,8 @@ public class OverlordClientImpl implements OverlordClient
|
|||
.jsonContent(jsonMapper, taskIds),
|
||||
new BytesFullResponseHandler()
|
||||
),
|
||||
holder -> deserialize(holder, new TypeReference<Map<String, TaskStatus>>() {})
|
||||
holder ->
|
||||
JacksonUtils.readValue(jsonMapper, holder.getContent(), new TypeReference<Map<String, TaskStatus>>() {})
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -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<Map<String, List<Interval>>> findLockedIntervals(Map<String, Integer> 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<String, List<Interval>> response = JacksonUtils.readValue(
|
||||
jsonMapper,
|
||||
holder.getContent(),
|
||||
new TypeReference<Map<String, List<Interval>>>() {}
|
||||
);
|
||||
|
||||
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<TaskPayloadResponse> taskPayload(String taskId)
|
||||
public ListenableFuture<CloseableIterator<SupervisorStatus>> 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<List<IndexingWorkerInfo>> 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<List<IndexingWorkerInfo>>() {})
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<IndexingTotalWorkerCapacityInfo> 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<Integer> 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<String, Object> 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<TaskPayloadResponse> 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<TaskPayloadResponse>()
|
||||
{
|
||||
})
|
||||
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> T deserialize(final BytesFullResponseHolder bytesHolder, final Class<T> clazz)
|
||||
private <T> JsonParserIterator<T> asJsonParserIterator(final InputStream in, final Class<T> clazz)
|
||||
{
|
||||
try {
|
||||
return jsonMapper.readValue(bytesHolder.getContent(), clazz);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private <T> T deserialize(final BytesFullResponseHolder bytesHolder, final TypeReference<T> 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
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String, LoadQueuePeon> 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<CompactSegments> compactSegmentsDutyFromCustomGroups = getCompactSegmentsDutyFromCustomGroups();
|
||||
if (compactSegmentsDutyFromCustomGroups.isEmpty()) {
|
||||
return new CompactSegments(config, compactionSegmentSearchPolicy, indexingServiceClient);
|
||||
return new CompactSegments(config, compactionSegmentSearchPolicy, overlordClient);
|
||||
} else {
|
||||
if (compactSegmentsDutyFromCustomGroups.size() > 1) {
|
||||
log.warn(
|
||||
|
|
|
@ -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<DateTime> 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<TaskStatusPlus> 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
|
||||
);
|
||||
|
|
|
@ -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<TaskStatusPlus> compactionTasks = filterNonCompactionTasks(indexingServiceClient.getActiveTasks());
|
||||
final CloseableIterator<TaskStatusPlus> activeTasks =
|
||||
FutureUtils.getUnchecked(overlordClient.taskStatuses(null, null, 0), true);
|
||||
final List<TaskStatusPlus> 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<String, List<Interval>> 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<TaskStatusPlus> filterNonCompactionTasks(List<TaskStatusPlus> taskStatuses)
|
||||
private static List<TaskStatusPlus> filterNonCompactionTasks(CloseableIterator<TaskStatusPlus> 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<TaskStatusPlus> 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<DataSegment> segments,
|
||||
int compactionTaskPriority,
|
||||
@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig,
|
||||
@Nullable ClientCompactionTaskGranularitySpec granularitySpec,
|
||||
@Nullable ClientCompactionTaskDimensionsSpec dimensionsSpec,
|
||||
@Nullable AggregatorFactory[] metricsSpec,
|
||||
@Nullable ClientCompactionTaskTransformSpec transformSpec,
|
||||
@Nullable Boolean dropExisting,
|
||||
@Nullable Map<String, Object> 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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<HttpClient> httpClientProvider;
|
||||
private final DruidHttpClientConfig httpClientConfig;
|
||||
|
||||
@Inject
|
||||
OverlordProxyServlet(
|
||||
@IndexingService DruidLeaderClient druidLeaderClient,
|
||||
OverlordClient overlordClient,
|
||||
@Global Provider<HttpClient> 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,
|
||||
|
|
|
@ -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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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);
|
||||
}
|
||||
}
|
||||
|
|
@ -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<URI> findCurrentLeader()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Void> runTask(String taskId, Object taskObject)
|
||||
{
|
||||
|
@ -41,6 +54,16 @@ public class NoopOverlordClient implements OverlordClient
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<CloseableIterator<TaskStatusPlus>> taskStatuses(
|
||||
@Nullable String state,
|
||||
@Nullable String dataSource,
|
||||
@Nullable Integer maxCompletedTasks
|
||||
)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Map<String, TaskStatus>> taskStatuses(Set<String> taskIds)
|
||||
{
|
||||
|
@ -53,6 +76,12 @@ public class NoopOverlordClient implements OverlordClient
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<TaskPayloadResponse> taskPayload(String taskId)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Map<String, Object>> taskReportAsMap(String taskId)
|
||||
{
|
||||
|
@ -60,7 +89,31 @@ public class NoopOverlordClient implements OverlordClient
|
|||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<TaskPayloadResponse> taskPayload(String taskId)
|
||||
public ListenableFuture<CloseableIterator<SupervisorStatus>> supervisorStatuses()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Map<String, List<Interval>>> findLockedIntervals(Map<String, Integer> minTaskPriority)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Integer> killPendingSegments(String dataSource, Interval interval)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<List<IndexingWorkerInfo>> getWorkers()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<IndexingTotalWorkerCapacityInfo> getTotalWorkerCapacity()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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<String, String> 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;
|
||||
|
|
|
@ -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<TaskStatusPlus> 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<String, Integer> priorityMap = ImmutableMap.of("foo", 3);
|
||||
final Map<String, List<Interval>> 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<String, Integer> 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<SupervisorStatus> 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<String, Object> 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<Map<String, Object>> 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<Map<String, Object>> 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<String, Object> 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<IndexingWorkerInfo> 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()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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<String> 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);
|
||||
}
|
||||
|
|
|
@ -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<TaskStatusPlus> it;
|
||||
|
||||
public TasksEnumerable(JsonParserIterator<TaskStatusPlus> tasks)
|
||||
public TasksEnumerable(CloseableIterator<TaskStatusPlus> 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<TaskStatusPlus> getAuthorizedTasks(
|
||||
JsonParserIterator<TaskStatusPlus> it,
|
||||
CloseableIterator<TaskStatusPlus> it,
|
||||
DataContext root
|
||||
)
|
||||
{
|
||||
|
@ -902,39 +909,20 @@ public class SystemSchema extends AbstractSchema
|
|||
|
||||
}
|
||||
|
||||
//Note that overlord must be up to get tasks
|
||||
private static JsonParserIterator<TaskStatusPlus> getTasks(
|
||||
DruidLeaderClient indexingServiceClient,
|
||||
ObjectMapper jsonMapper
|
||||
)
|
||||
{
|
||||
return getThingsFromLeaderNode(
|
||||
"/druid/indexer/v1/tasks",
|
||||
new TypeReference<TaskStatusPlus>()
|
||||
{
|
||||
},
|
||||
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<SupervisorStatus> it;
|
||||
|
||||
public SupervisorsEnumerable(JsonParserIterator<SupervisorStatus> tasks)
|
||||
public SupervisorsEnumerable(CloseableIterator<SupervisorStatus> 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<SupervisorStatus> getAuthorizedSupervisors(
|
||||
JsonParserIterator<SupervisorStatus> it,
|
||||
CloseableIterator<SupervisorStatus> 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<SupervisorStatus> getSupervisors(
|
||||
DruidLeaderClient indexingServiceClient,
|
||||
ObjectMapper jsonMapper
|
||||
)
|
||||
{
|
||||
return getThingsFromLeaderNode(
|
||||
"/druid/indexer/v1/supervisor?system",
|
||||
new TypeReference<SupervisorStatus>()
|
||||
{
|
||||
},
|
||||
indexingServiceClient,
|
||||
jsonMapper
|
||||
);
|
||||
}
|
||||
|
||||
public static <T> JsonParserIterator<T> getThingsFromLeaderNode(
|
||||
String query,
|
||||
TypeReference<T> typeRef,
|
||||
|
@ -1104,7 +1075,7 @@ public class SystemSchema extends AbstractSchema
|
|||
);
|
||||
}
|
||||
|
||||
private static <T> CloseableIterator<T> wrap(Iterator<T> iterator, JsonParserIterator<T> it)
|
||||
private static <T> CloseableIterator<T> wrap(Iterator<T> iterator, Closeable closer)
|
||||
{
|
||||
return new CloseableIterator<T>()
|
||||
{
|
||||
|
@ -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();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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<DruidServer> 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<List<TaskStatusPlus>>() {}).iterator()
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
EasyMock.replay(overlordClient, request, responseHandler);
|
||||
DataContext dataContext = createDataContext(Users.SUPER);
|
||||
final List<Object[]> 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<List<TaskStatusPlus>>() {}).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<Object[]> 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<List<SupervisorStatus>>() {}).iterator()
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
EasyMock.replay(client, request, responseHandler);
|
||||
EasyMock.replay(overlordClient);
|
||||
DataContext dataContext = createDataContext(Users.SUPER);
|
||||
final List<Object[]> 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<List<SupervisorStatus>>() {}).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<Object[]> rows = supervisorTable
|
||||
|
|
|
@ -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<URI> 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()
|
||||
);
|
||||
|
|
|
@ -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()
|
||||
|
|
Loading…
Reference in New Issue