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:
Gian Merlino 2023-07-24 21:14:27 -07:00 committed by GitHub
parent f742bb7376
commit 2f9619a96f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
51 changed files with 2111 additions and 2096 deletions

View File

@ -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);
}

View File

@ -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()
);
}
}

View File

@ -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)
{

View File

@ -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;
}
}

View File

@ -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.
*/

View File

@ -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;

View File

@ -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);

View File

@ -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())

View File

@ -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());
}
}

View File

@ -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")
)
);
}
}

View File

@ -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(),

View File

@ -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;
}
}

View File

@ -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)
);
}

View File

@ -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()
{

View File

@ -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
{

View File

@ -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

View File

@ -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);

View File

@ -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;

View File

@ -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

View File

@ -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);
}
}
}

View File

@ -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);
}

View File

@ -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 +
'}';
}
}

View File

@ -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 + '\'' +
'}';
}
}

View File

@ -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 +
'}';
}
}

View File

@ -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"
);
}
}

View File

@ -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(),

View File

@ -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);
}

View File

@ -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
);
}
}

View File

@ -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(

View File

@ -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
);

View File

@ -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;
}
}

View File

@ -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) {

View File

@ -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) {

View File

@ -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,

View File

@ -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);
}
}

View File

@ -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();
}

View File

@ -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();
}
}

View File

@ -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();
}
}

View File

@ -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;

View File

@ -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()
);
}
}

View File

@ -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)

View File

@ -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

View File

@ -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&param2=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&param2=test2", uri.toString());
}
EasyMock.verify(overlordClient, request);
}
}

View File

@ -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);

View File

@ -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);
}

View File

@ -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();
}
};
}

View File

@ -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);

View File

@ -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

View File

@ -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()
);

View File

@ -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()