mirror of https://github.com/apache/druid.git
make the selection strategy in rtr extendable
This commit is contained in:
parent
7cfe86b7bc
commit
4a09678739
|
@ -25,14 +25,11 @@ import com.google.common.base.Joiner;
|
|||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Stopwatch;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
@ -49,7 +46,7 @@ import io.druid.curator.cache.PathChildrenCacheFactory;
|
|||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import io.druid.indexing.overlord.setup.WorkerSelectStrategy;
|
||||
import io.druid.indexing.worker.TaskAnnouncement;
|
||||
import io.druid.indexing.worker.Worker;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
|
@ -70,10 +67,8 @@ import java.io.InputStream;
|
|||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
@ -109,8 +104,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
private final CuratorFramework cf;
|
||||
private final PathChildrenCacheFactory pathChildrenCacheFactory;
|
||||
private final PathChildrenCache workerPathCache;
|
||||
private final Supplier<WorkerSetupData> workerSetupData;
|
||||
private final HttpClient httpClient;
|
||||
private final WorkerSelectStrategy strategy;
|
||||
|
||||
// all workers that exist in ZK
|
||||
private final ConcurrentMap<String, ZkWorker> zkWorkers = new ConcurrentHashMap<>();
|
||||
|
@ -135,8 +130,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
ZkPathsConfig zkPaths,
|
||||
CuratorFramework cf,
|
||||
PathChildrenCacheFactory pathChildrenCacheFactory,
|
||||
Supplier<WorkerSetupData> workerSetupData,
|
||||
HttpClient httpClient
|
||||
HttpClient httpClient,
|
||||
WorkerSelectStrategy strategy
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
@ -145,8 +140,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
this.cf = cf;
|
||||
this.pathChildrenCacheFactory = pathChildrenCacheFactory;
|
||||
this.workerPathCache = pathChildrenCacheFactory.make(cf, zkPaths.getIndexerAnnouncementPath());
|
||||
this.workerSetupData = workerSetupData;
|
||||
this.httpClient = httpClient;
|
||||
this.strategy = strategy;
|
||||
}
|
||||
|
||||
@LifecycleStart
|
||||
|
@ -524,9 +519,9 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
return true;
|
||||
} else {
|
||||
// Nothing running this task, announce it in ZK for a worker to run it
|
||||
ZkWorker zkWorker = findWorkerForTask(task);
|
||||
if (zkWorker != null) {
|
||||
announceTask(task, zkWorker, taskRunnerWorkItem);
|
||||
Optional<ZkWorker> zkWorker = strategy.findWorkerForTask(zkWorkers, task);
|
||||
if (zkWorker.isPresent()) {
|
||||
announceTask(task, zkWorker.get(), taskRunnerWorkItem);
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
|
@ -789,37 +784,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
}
|
||||
}
|
||||
|
||||
private ZkWorker findWorkerForTask(final Task task)
|
||||
{
|
||||
TreeSet<ZkWorker> sortedWorkers = Sets.newTreeSet(
|
||||
new Comparator<ZkWorker>()
|
||||
{
|
||||
@Override
|
||||
public int compare(
|
||||
ZkWorker zkWorker, ZkWorker zkWorker2
|
||||
)
|
||||
{
|
||||
int retVal = Ints.compare(zkWorker2.getCurrCapacityUsed(), zkWorker.getCurrCapacityUsed());
|
||||
if (retVal == 0) {
|
||||
retVal = zkWorker.getWorker().getHost().compareTo(zkWorker2.getWorker().getHost());
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
);
|
||||
sortedWorkers.addAll(zkWorkers.values());
|
||||
final String minWorkerVer = config.getMinWorkerVersion();
|
||||
|
||||
for (ZkWorker zkWorker : sortedWorkers) {
|
||||
if (zkWorker.canRunTask(task) && zkWorker.isValidVersion(minWorkerVer)) {
|
||||
return zkWorker;
|
||||
}
|
||||
}
|
||||
log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values());
|
||||
return null;
|
||||
}
|
||||
|
||||
private void taskComplete(
|
||||
RemoteTaskRunnerWorkItem taskRunnerWorkItem,
|
||||
ZkWorker zkWorker,
|
||||
|
|
|
@ -20,26 +20,25 @@
|
|||
package io.druid.indexing.overlord;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
import io.druid.curator.cache.SimplePathChildrenCacheFactory;
|
||||
import io.druid.guice.annotations.Global;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import io.druid.indexing.overlord.setup.WorkerSelectStrategy;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
|
||||
/**
|
||||
*/
|
||||
*/
|
||||
public class RemoteTaskRunnerFactory implements TaskRunnerFactory
|
||||
{
|
||||
private final CuratorFramework curator;
|
||||
private final RemoteTaskRunnerConfig remoteTaskRunnerConfig;
|
||||
private final ZkPathsConfig zkPaths;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final Supplier<WorkerSetupData> setupDataWatch;
|
||||
private final HttpClient httpClient;
|
||||
private final WorkerSelectStrategy strategy;
|
||||
|
||||
@Inject
|
||||
public RemoteTaskRunnerFactory(
|
||||
|
@ -47,15 +46,16 @@ public class RemoteTaskRunnerFactory implements TaskRunnerFactory
|
|||
final RemoteTaskRunnerConfig remoteTaskRunnerConfig,
|
||||
final ZkPathsConfig zkPaths,
|
||||
final ObjectMapper jsonMapper,
|
||||
final Supplier<WorkerSetupData> setupDataWatch,
|
||||
@Global final HttpClient httpClient
|
||||
) {
|
||||
@Global final HttpClient httpClient,
|
||||
final WorkerSelectStrategy strategy
|
||||
)
|
||||
{
|
||||
this.curator = curator;
|
||||
this.remoteTaskRunnerConfig = remoteTaskRunnerConfig;
|
||||
this.zkPaths = zkPaths;
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.setupDataWatch = setupDataWatch;
|
||||
this.httpClient = httpClient;
|
||||
this.strategy = strategy;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -70,8 +70,8 @@ public class RemoteTaskRunnerFactory implements TaskRunnerFactory
|
|||
.Builder()
|
||||
.withCompressed(remoteTaskRunnerConfig.isCompressZnodes())
|
||||
.build(),
|
||||
setupDataWatch,
|
||||
httpClient
|
||||
httpClient,
|
||||
strategy
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexing.overlord.setup;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.overlord.ZkWorker;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FillCapacityWorkerSelectStrategy implements WorkerSelectStrategy
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(FillCapacityWorkerSelectStrategy.class);
|
||||
|
||||
private final RemoteTaskRunnerConfig config;
|
||||
|
||||
@Inject
|
||||
public FillCapacityWorkerSelectStrategy(RemoteTaskRunnerConfig config)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
public Optional<ZkWorker> findWorkerForTask(final Map<String, ZkWorker> zkWorkers, final Task task)
|
||||
{
|
||||
TreeSet<ZkWorker> sortedWorkers = Sets.newTreeSet(
|
||||
new Comparator<ZkWorker>()
|
||||
{
|
||||
@Override
|
||||
public int compare(
|
||||
ZkWorker zkWorker, ZkWorker zkWorker2
|
||||
)
|
||||
{
|
||||
int retVal = Ints.compare(zkWorker2.getCurrCapacityUsed(), zkWorker.getCurrCapacityUsed());
|
||||
if (retVal == 0) {
|
||||
retVal = zkWorker.getWorker().getHost().compareTo(zkWorker2.getWorker().getHost());
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
);
|
||||
sortedWorkers.addAll(zkWorkers.values());
|
||||
final String minWorkerVer = config.getMinWorkerVersion();
|
||||
|
||||
for (ZkWorker zkWorker : sortedWorkers) {
|
||||
if (zkWorker.canRunTask(task) && zkWorker.isValidVersion(minWorkerVer)) {
|
||||
return Optional.of(zkWorker);
|
||||
}
|
||||
}
|
||||
log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values());
|
||||
|
||||
return Optional.absent();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexing.overlord.setup;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.google.common.base.Optional;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.overlord.ZkWorker;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "fillCapacity", value = FillCapacityWorkerSelectStrategy.class)
|
||||
})
|
||||
public interface WorkerSelectStrategy
|
||||
{
|
||||
public Optional<ZkWorker> findWorkerForTask(final Map<String, ZkWorker> zkWorkers, final Task task);
|
||||
}
|
|
@ -39,6 +39,8 @@ import io.druid.indexing.common.TestRealtimeTask;
|
|||
import io.druid.indexing.common.TestUtils;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.common.task.TaskResource;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy;
|
||||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import io.druid.indexing.worker.TaskAnnouncement;
|
||||
import io.druid.indexing.worker.Worker;
|
||||
|
@ -367,9 +369,10 @@ public class RemoteTaskRunnerTest
|
|||
|
||||
private void makeRemoteTaskRunner() throws Exception
|
||||
{
|
||||
RemoteTaskRunnerConfig config = new TestRemoteTaskRunnerConfig();
|
||||
remoteTaskRunner = new RemoteTaskRunner(
|
||||
jsonMapper,
|
||||
new TestRemoteTaskRunnerConfig(),
|
||||
config,
|
||||
new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
|
@ -380,8 +383,8 @@ public class RemoteTaskRunnerTest
|
|||
},
|
||||
cf,
|
||||
new SimplePathChildrenCacheFactory.Builder().build(),
|
||||
DSuppliers.of(new AtomicReference<WorkerSetupData>(new WorkerSetupData(0, 1, null, null, null))),
|
||||
null
|
||||
null,
|
||||
new FillCapacityWorkerSelectStrategy(config)
|
||||
);
|
||||
|
||||
remoteTaskRunner.start();
|
||||
|
|
|
@ -70,6 +70,8 @@ import io.druid.indexing.overlord.scaling.ResourceManagementSchedulerFactoryImpl
|
|||
import io.druid.indexing.overlord.scaling.ResourceManagementStrategy;
|
||||
import io.druid.indexing.overlord.scaling.SimpleResourceManagementConfig;
|
||||
import io.druid.indexing.overlord.scaling.SimpleResourceManagementStrategy;
|
||||
import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy;
|
||||
import io.druid.indexing.overlord.setup.WorkerSelectStrategy;
|
||||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import io.druid.indexing.worker.config.WorkerConfig;
|
||||
import io.druid.segment.realtime.firehose.ChatHandlerProvider;
|
||||
|
@ -196,6 +198,20 @@ public class CliOverlord extends ServerRunnable
|
|||
|
||||
biddy.addBinding("remote").to(RemoteTaskRunnerFactory.class).in(LazySingleton.class);
|
||||
binder.bind(RemoteTaskRunnerFactory.class).in(LazySingleton.class);
|
||||
|
||||
PolyBind.createChoice(
|
||||
binder,
|
||||
"druid.indexer.runner.workerSelectStrategy.type",
|
||||
Key.get(WorkerSelectStrategy.class),
|
||||
Key.get(FillCapacityWorkerSelectStrategy.class)
|
||||
);
|
||||
final MapBinder<String, WorkerSelectStrategy> stratBinder = PolyBind.optionBinder(
|
||||
binder,
|
||||
Key.get(WorkerSelectStrategy.class)
|
||||
);
|
||||
|
||||
stratBinder.addBinding("fillCapacity").to(FillCapacityWorkerSelectStrategy.class);
|
||||
binder.bind(FillCapacityWorkerSelectStrategy.class).in(LazySingleton.class);
|
||||
}
|
||||
|
||||
private void configureAutoscale(Binder binder)
|
||||
|
|
Loading…
Reference in New Issue