mirror of https://github.com/apache/druid.git
Merge pull request #998 from drcrallen/zkConfigJackson
Change zk configs to use Jackson injection instead of Skife
This commit is contained in:
commit
ab0ddc0d7c
|
@ -22,6 +22,7 @@ package io.druid.guice;
|
|||
import com.google.inject.Binder;
|
||||
import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
import io.druid.server.initialization.IndexerZkConfig;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -31,5 +32,6 @@ public class IndexingServiceModuleHelper
|
|||
{
|
||||
JsonConfigProvider.bind(binder, "druid.indexer.runner", ForkingTaskRunnerConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.indexer.runner", RemoteTaskRunnerConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.zk.paths.indexer", IndexerZkConfig.class);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,6 +51,7 @@ import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
|||
import io.druid.indexing.overlord.setup.WorkerSelectStrategy;
|
||||
import io.druid.indexing.worker.TaskAnnouncement;
|
||||
import io.druid.indexing.worker.Worker;
|
||||
import io.druid.server.initialization.IndexerZkConfig;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import io.druid.tasklogs.TaskLogStreamer;
|
||||
import org.apache.commons.lang.mutable.MutableInt;
|
||||
|
@ -102,7 +103,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final RemoteTaskRunnerConfig config;
|
||||
private final ZkPathsConfig zkPaths;
|
||||
private final IndexerZkConfig indexerZkConfig;
|
||||
private final CuratorFramework cf;
|
||||
private final PathChildrenCacheFactory pathChildrenCacheFactory;
|
||||
private final PathChildrenCache workerPathCache;
|
||||
|
@ -129,7 +130,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
public RemoteTaskRunner(
|
||||
ObjectMapper jsonMapper,
|
||||
RemoteTaskRunnerConfig config,
|
||||
ZkPathsConfig zkPaths,
|
||||
IndexerZkConfig indexerZkConfig,
|
||||
CuratorFramework cf,
|
||||
PathChildrenCacheFactory pathChildrenCacheFactory,
|
||||
HttpClient httpClient,
|
||||
|
@ -138,10 +139,10 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.config = config;
|
||||
this.zkPaths = zkPaths;
|
||||
this.indexerZkConfig = indexerZkConfig;
|
||||
this.cf = cf;
|
||||
this.pathChildrenCacheFactory = pathChildrenCacheFactory;
|
||||
this.workerPathCache = pathChildrenCacheFactory.make(cf, zkPaths.getIndexerAnnouncementPath());
|
||||
this.workerPathCache = pathChildrenCacheFactory.make(cf, indexerZkConfig.getAnnouncementsPath());
|
||||
this.httpClient = httpClient;
|
||||
this.strategy = strategy;
|
||||
}
|
||||
|
@ -496,7 +497,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
} else {
|
||||
final String workerId = worker.getHost();
|
||||
log.info("Cleaning up task[%s] on worker[%s]", taskId, workerId);
|
||||
final String statusPath = JOINER.join(zkPaths.getIndexerStatusPath(), workerId, taskId);
|
||||
final String statusPath = JOINER.join(indexerZkConfig.getStatus(), workerId, taskId);
|
||||
try {
|
||||
cf.delete().guaranteed().forPath(statusPath);
|
||||
}
|
||||
|
@ -582,7 +583,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxZnodeBytes());
|
||||
}
|
||||
|
||||
String taskPath = JOINER.join(zkPaths.getIndexerTaskPath(), theWorker.getHost(), task.getId());
|
||||
String taskPath = JOINER.join(indexerZkConfig.getTasksPath(), theWorker.getHost(), task.getId());
|
||||
|
||||
if (cf.checkExists().forPath(taskPath) == null) {
|
||||
cf.create()
|
||||
|
@ -642,7 +643,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
log.info("Worker[%s] reportin' for duty!", worker.getHost());
|
||||
|
||||
try {
|
||||
final String workerStatusPath = JOINER.join(zkPaths.getIndexerStatusPath(), worker.getHost());
|
||||
final String workerStatusPath = JOINER.join(indexerZkConfig.getStatus(), worker.getHost());
|
||||
final PathChildrenCache statusCache = pathChildrenCacheFactory.make(cf, workerStatusPath);
|
||||
final SettableFuture<ZkWorker> retVal = SettableFuture.create();
|
||||
final ZkWorker zkWorker = new ZkWorker(
|
||||
|
@ -787,7 +788,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
if (zkWorker != null) {
|
||||
try {
|
||||
List<String> tasksToFail = Lists.newArrayList(
|
||||
cf.getChildren().forPath(JOINER.join(zkPaths.getIndexerTaskPath(), worker.getHost()))
|
||||
cf.getChildren().forPath(JOINER.join(indexerZkConfig.getTasksPath(), worker.getHost()))
|
||||
);
|
||||
log.info("[%s]: Found %d tasks assigned", worker.getHost(), tasksToFail.size());
|
||||
|
||||
|
@ -805,7 +806,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
for (String assignedTask : tasksToFail) {
|
||||
RemoteTaskRunnerWorkItem taskRunnerWorkItem = runningTasks.remove(assignedTask);
|
||||
if (taskRunnerWorkItem != null) {
|
||||
String taskPath = JOINER.join(zkPaths.getIndexerTaskPath(), worker.getHost(), assignedTask);
|
||||
String taskPath = JOINER.join(indexerZkConfig.getTasksPath(), worker.getHost(), assignedTask);
|
||||
if (cf.checkExists().forPath(taskPath) != null) {
|
||||
cf.delete().guaranteed().forPath(taskPath);
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
|||
import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy;
|
||||
import io.druid.indexing.overlord.setup.WorkerBehaviorConfig;
|
||||
import io.druid.indexing.overlord.setup.WorkerSelectStrategy;
|
||||
import io.druid.server.initialization.IndexerZkConfig;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
|
||||
|
@ -38,7 +39,7 @@ public class RemoteTaskRunnerFactory implements TaskRunnerFactory
|
|||
{
|
||||
private final CuratorFramework curator;
|
||||
private final RemoteTaskRunnerConfig remoteTaskRunnerConfig;
|
||||
private final ZkPathsConfig zkPaths;
|
||||
private final IndexerZkConfig zkPaths;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final HttpClient httpClient;
|
||||
private final WorkerSelectStrategy strategy;
|
||||
|
@ -47,7 +48,7 @@ public class RemoteTaskRunnerFactory implements TaskRunnerFactory
|
|||
public RemoteTaskRunnerFactory(
|
||||
final CuratorFramework curator,
|
||||
final RemoteTaskRunnerConfig remoteTaskRunnerConfig,
|
||||
final ZkPathsConfig zkPaths,
|
||||
final IndexerZkConfig zkPaths,
|
||||
final ObjectMapper jsonMapper,
|
||||
@Global final HttpClient httpClient,
|
||||
final Supplier<WorkerBehaviorConfig> workerBehaviourConfigSupplier
|
||||
|
|
|
@ -38,6 +38,7 @@ import io.druid.indexing.overlord.config.TaskQueueConfig;
|
|||
import io.druid.indexing.overlord.autoscaling.ResourceManagementScheduler;
|
||||
import io.druid.indexing.overlord.autoscaling.ResourceManagementSchedulerFactory;
|
||||
import io.druid.server.DruidNode;
|
||||
import io.druid.server.initialization.IndexerZkConfig;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.curator.framework.recipes.leader.LeaderSelector;
|
||||
|
@ -74,7 +75,7 @@ public class TaskMaster
|
|||
final TaskStorage taskStorage,
|
||||
final TaskActionClientFactory taskActionClientFactory,
|
||||
@Self final DruidNode node,
|
||||
final ZkPathsConfig zkPaths,
|
||||
final IndexerZkConfig zkPaths,
|
||||
final TaskRunnerFactory runnerFactory,
|
||||
final ResourceManagementSchedulerFactory managementSchedulerFactory,
|
||||
final CuratorFramework curator,
|
||||
|
@ -85,7 +86,7 @@ public class TaskMaster
|
|||
this.taskActionClientFactory = taskActionClientFactory;
|
||||
this.leaderSelector = new LeaderSelector(
|
||||
curator,
|
||||
zkPaths.getIndexerLeaderLatchPath(),
|
||||
zkPaths.getLeaderLatchPath(),
|
||||
new LeaderSelectorListener()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -31,7 +31,7 @@ import com.metamx.common.lifecycle.LifecycleStop;
|
|||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.curator.announcement.Announcer;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import io.druid.server.initialization.IndexerZkConfig;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -63,7 +63,7 @@ public class WorkerCuratorCoordinator
|
|||
@Inject
|
||||
public WorkerCuratorCoordinator(
|
||||
ObjectMapper jsonMapper,
|
||||
ZkPathsConfig zkPaths,
|
||||
IndexerZkConfig indexerZkConfig,
|
||||
RemoteTaskRunnerConfig config,
|
||||
CuratorFramework curatorFramework,
|
||||
Worker worker
|
||||
|
@ -76,9 +76,9 @@ public class WorkerCuratorCoordinator
|
|||
|
||||
this.announcer = new Announcer(curatorFramework, MoreExecutors.sameThreadExecutor());
|
||||
|
||||
this.baseAnnouncementsPath = getPath(Arrays.asList(zkPaths.getIndexerAnnouncementPath(), worker.getHost()));
|
||||
this.baseTaskPath = getPath(Arrays.asList(zkPaths.getIndexerTaskPath(), worker.getHost()));
|
||||
this.baseStatusPath = getPath(Arrays.asList(zkPaths.getIndexerStatusPath(), worker.getHost()));
|
||||
this.baseAnnouncementsPath = getPath(Arrays.asList(indexerZkConfig.getAnnouncementsPath(), worker.getHost()));
|
||||
this.baseTaskPath = getPath(Arrays.asList(indexerZkConfig.getTasksPath(), worker.getHost()));
|
||||
this.baseStatusPath = getPath(Arrays.asList(indexerZkConfig.getStatus(), worker.getHost()));
|
||||
}
|
||||
|
||||
@LifecycleStart
|
||||
|
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013, 2014 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.server.initialization;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.curator.utils.ZKPaths;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class IndexerZkConfig
|
||||
{
|
||||
@Inject
|
||||
@JsonIgnore
|
||||
private ZkPathsConfig zkPathsConfig = new ZkPathsConfig();
|
||||
@JsonProperty
|
||||
private String base;
|
||||
@JsonProperty
|
||||
private String announcementsPath;
|
||||
@JsonProperty
|
||||
private String tasksPath;
|
||||
@JsonProperty
|
||||
private String status;
|
||||
@JsonProperty
|
||||
private String leaderLatchPath;
|
||||
|
||||
private String defaultIndexerPath(final String subPath)
|
||||
{
|
||||
return getZkPathsConfig().defaultPath(ZKPaths.makePath(getBase(), subPath));
|
||||
}
|
||||
|
||||
public String getBase()
|
||||
{
|
||||
return base == null ? "indexer" : base;
|
||||
}
|
||||
|
||||
public String getAnnouncementsPath()
|
||||
{
|
||||
return announcementsPath == null ? defaultIndexerPath("announcements") : announcementsPath;
|
||||
}
|
||||
|
||||
public String getTasksPath()
|
||||
{
|
||||
return tasksPath == null ? defaultIndexerPath("tasks") : tasksPath;
|
||||
}
|
||||
|
||||
public String getStatus()
|
||||
{
|
||||
return status == null ? defaultIndexerPath("status") : status;
|
||||
}
|
||||
|
||||
public String getLeaderLatchPath()
|
||||
{
|
||||
return leaderLatchPath == null ? defaultIndexerPath("leaderLatchPath") : leaderLatchPath;
|
||||
}
|
||||
|
||||
public ZkPathsConfig getZkPathsConfig()
|
||||
{
|
||||
return zkPathsConfig;
|
||||
}
|
||||
|
||||
// Setter required for easy debugging
|
||||
public IndexerZkConfig setZkPathsConfig(ZkPathsConfig zkPathsConfig)
|
||||
{
|
||||
this.zkPathsConfig = zkPathsConfig;
|
||||
return this;
|
||||
}
|
||||
}
|
|
@ -43,6 +43,7 @@ import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy;
|
|||
import io.druid.indexing.worker.TaskAnnouncement;
|
||||
import io.druid.indexing.worker.Worker;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.server.initialization.IndexerZkConfig;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.curator.framework.CuratorFrameworkFactory;
|
||||
|
@ -57,6 +58,7 @@ import org.junit.Test;
|
|||
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class RemoteTaskRunnerTest
|
||||
{
|
||||
|
@ -66,6 +68,7 @@ public class RemoteTaskRunnerTest
|
|||
private static final String announcementsPath = String.format("%s/indexer/announcements/worker", basePath);
|
||||
private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath);
|
||||
private static final String statusPath = String.format("%s/indexer/status/worker", basePath);
|
||||
private static final int TIMEOUT_SECONDS = 5;
|
||||
|
||||
private TestingCluster testingCluster;
|
||||
private CuratorFramework cf;
|
||||
|
@ -282,7 +285,7 @@ public class RemoteTaskRunnerTest
|
|||
|
||||
cf.delete().forPath(joiner.join(statusPath, task.getId()));
|
||||
|
||||
TaskStatus status = future.get();
|
||||
TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
|
||||
|
||||
Assert.assertEquals(status.getStatusCode(), TaskStatus.Status.FAILED);
|
||||
}
|
||||
|
@ -335,7 +338,7 @@ public class RemoteTaskRunnerTest
|
|||
|
||||
ListenableFuture<TaskStatus> future = remoteTaskRunner.run(task);
|
||||
|
||||
TaskStatus status = future.get();
|
||||
TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
|
||||
|
||||
Assert.assertEquals(TaskStatus.Status.SUCCESS, status.getStatusCode());
|
||||
}
|
||||
|
@ -353,7 +356,7 @@ public class RemoteTaskRunnerTest
|
|||
|
||||
cf.delete().forPath(announcementsPath);
|
||||
|
||||
TaskStatus status = future.get();
|
||||
TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
|
||||
|
||||
Assert.assertEquals(TaskStatus.Status.FAILED, status.getStatusCode());
|
||||
}
|
||||
|
@ -393,14 +396,14 @@ public class RemoteTaskRunnerTest
|
|||
remoteTaskRunner = new RemoteTaskRunner(
|
||||
jsonMapper,
|
||||
config,
|
||||
new ZkPathsConfig()
|
||||
new IndexerZkConfig().setZkPathsConfig(new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return basePath;
|
||||
}
|
||||
},
|
||||
}),
|
||||
cf,
|
||||
new SimplePathChildrenCacheFactory.Builder().build(),
|
||||
null,
|
||||
|
|
|
@ -50,6 +50,7 @@ import io.druid.segment.loading.LocalDataSegmentPuller;
|
|||
import io.druid.segment.loading.OmniSegmentLoader;
|
||||
import io.druid.segment.loading.SegmentLoaderConfig;
|
||||
import io.druid.segment.loading.StorageLocationConfig;
|
||||
import io.druid.server.initialization.IndexerZkConfig;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import junit.framework.Assert;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
|
@ -139,14 +140,15 @@ public class WorkerTaskMonitorTest
|
|||
|
||||
workerCuratorCoordinator = new WorkerCuratorCoordinator(
|
||||
jsonMapper,
|
||||
new IndexerZkConfig().setZkPathsConfig(
|
||||
new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return basePath;
|
||||
}
|
||||
},
|
||||
}),
|
||||
new TestRemoteTaskRunnerConfig(),
|
||||
cf,
|
||||
worker
|
||||
|
|
|
@ -25,6 +25,7 @@ import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
|||
import io.druid.indexing.worker.Worker;
|
||||
import io.druid.indexing.worker.WorkerCuratorCoordinator;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.server.initialization.IndexerZkConfig;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import junit.framework.Assert;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
|
@ -76,14 +77,14 @@ public class WorkerResourceTest
|
|||
|
||||
curatorCoordinator = new WorkerCuratorCoordinator(
|
||||
jsonMapper,
|
||||
new ZkPathsConfig()
|
||||
new IndexerZkConfig().setZkPathsConfig(new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return basePath;
|
||||
}
|
||||
},
|
||||
}),
|
||||
new RemoteTaskRunnerConfig(),
|
||||
cf,
|
||||
worker
|
||||
|
|
|
@ -0,0 +1,175 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013, 2014 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.server.initialization;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.name.Names;
|
||||
import io.druid.curator.CuratorConfig;
|
||||
import io.druid.guice.GuiceInjectors;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.JsonConfigurator;
|
||||
import io.druid.initialization.Initialization;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.lang.reflect.Field;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class IndexerZkConfigTest
|
||||
{
|
||||
private static final String indexerPropertyString = "test.druid.zk.paths.indexer";
|
||||
private static final String zkServiceConfigString = "test.druid.zk.paths";
|
||||
private static final Collection<String> clobberableProperties = new ArrayList<>();
|
||||
|
||||
private static final Module simpleZkConfigModule = new Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/test");
|
||||
binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0);
|
||||
// See IndexingServiceModuleHelper
|
||||
JsonConfigProvider.bind(binder, indexerPropertyString, IndexerZkConfig.class);
|
||||
JsonConfigProvider.bind(
|
||||
binder, zkServiceConfigString,
|
||||
CuratorConfig.class
|
||||
);
|
||||
}
|
||||
};
|
||||
|
||||
private static final Map<String, String> priorValues = new HashMap<>();
|
||||
|
||||
@BeforeClass
|
||||
public static void setup()
|
||||
{
|
||||
for (Field field : IndexerZkConfig.class.getDeclaredFields()) {
|
||||
if (null != field.getAnnotation(JsonProperty.class)) {
|
||||
clobberableProperties.add(String.format("%s.%s", indexerPropertyString, field.getName()));
|
||||
}
|
||||
}
|
||||
for (Field field : ZkPathsConfig.class.getDeclaredFields()) {
|
||||
if (null != field.getAnnotation(JsonProperty.class)) {
|
||||
clobberableProperties.add(String.format("%s.%s", zkServiceConfigString, field.getName()));
|
||||
}
|
||||
}
|
||||
for (String clobberableProperty : clobberableProperties) {
|
||||
priorValues.put(clobberableProperty, System.getProperty(clobberableProperty));
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanup()
|
||||
{
|
||||
for (Map.Entry<String, String> entry : priorValues.entrySet()) {
|
||||
if (null != entry.getKey() && null != entry.getValue()) {
|
||||
System.setProperty(entry.getKey(), entry.getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, String> propertyValues = new HashMap<>();
|
||||
private int assertions = 0;
|
||||
|
||||
@Before
|
||||
public void setupTest()
|
||||
{
|
||||
for (String property : clobberableProperties) {
|
||||
propertyValues.put(property, UUID.randomUUID().toString());
|
||||
}
|
||||
System.getProperties().putAll(propertyValues);
|
||||
assertions = 0;
|
||||
}
|
||||
|
||||
|
||||
private void validateEntries(ZkPathsConfig zkPathsConfig)
|
||||
throws IllegalAccessException, NoSuchMethodException, InvocationTargetException
|
||||
{
|
||||
for (Field field : ZkPathsConfig.class.getDeclaredFields()) {
|
||||
if (null != field.getAnnotation(JsonProperty.class)) {
|
||||
String property = String.format("%s.%s", zkServiceConfigString, field.getName());
|
||||
String getter = String.format(
|
||||
"get%s%s",
|
||||
field.getName().substring(0, 1).toUpperCase(),
|
||||
field.getName().substring(1)
|
||||
);
|
||||
Method method = ZkPathsConfig.class.getDeclaredMethod(getter);
|
||||
Assert.assertEquals(propertyValues.get(property), method.invoke(zkPathsConfig));
|
||||
++assertions;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void validateEntries(IndexerZkConfig indexerZkConfig)
|
||||
throws IllegalAccessException, NoSuchMethodException, InvocationTargetException
|
||||
{
|
||||
for (Field field : IndexerZkConfig.class.getDeclaredFields()) {
|
||||
if (null != field.getAnnotation(JsonProperty.class)) {
|
||||
String property = String.format("%s.%s", indexerPropertyString, field.getName());
|
||||
String getter = String.format(
|
||||
"get%s%s",
|
||||
field.getName().substring(0, 1).toUpperCase(),
|
||||
field.getName().substring(1)
|
||||
);
|
||||
Method method = IndexerZkConfig.class.getDeclaredMethod(getter);
|
||||
Assert.assertEquals(propertyValues.get(property), method.invoke(indexerZkConfig));
|
||||
++assertions;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleConfig() throws IllegalAccessException, NoSuchMethodException, InvocationTargetException
|
||||
{
|
||||
final Injector injector = Initialization.makeInjectorWithModules(
|
||||
GuiceInjectors.makeStartupInjector(),
|
||||
ImmutableList.<Object>of(simpleZkConfigModule)
|
||||
);
|
||||
JsonConfigurator configurator = injector.getBinding(JsonConfigurator.class).getProvider().get();
|
||||
|
||||
JsonConfigProvider<ZkPathsConfig> zkPathsConfig = JsonConfigProvider.of(zkServiceConfigString, ZkPathsConfig.class);
|
||||
zkPathsConfig.inject(System.getProperties(), configurator);
|
||||
|
||||
JsonConfigProvider<IndexerZkConfig> indexerZkConfig = JsonConfigProvider.of(
|
||||
indexerPropertyString,
|
||||
IndexerZkConfig.class
|
||||
);
|
||||
indexerZkConfig.inject(System.getProperties(), configurator);
|
||||
|
||||
validateEntries(indexerZkConfig.get().get());
|
||||
validateEntries(zkPathsConfig.get().get());
|
||||
Assert.assertEquals(clobberableProperties.size(), assertions);
|
||||
}
|
||||
}
|
2
pom.xml
2
pom.xml
|
@ -456,7 +456,7 @@
|
|||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
<version>3.0</version>
|
||||
<version>3.3</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
|
|
|
@ -19,22 +19,50 @@
|
|||
|
||||
package io.druid.curator;
|
||||
|
||||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import javax.validation.constraints.Min;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class CuratorConfig
|
||||
public class CuratorConfig
|
||||
{
|
||||
@Config("druid.zk.service.host")
|
||||
@Default("localhost")
|
||||
public abstract String getZkHosts();
|
||||
@JsonProperty("host")
|
||||
private String zkHosts = "localhost";
|
||||
|
||||
@Config("druid.zk.service.sessionTimeoutMs")
|
||||
@Default("30000")
|
||||
public abstract int getZkSessionTimeoutMs();
|
||||
@JsonProperty("sessionTimeoutMs")
|
||||
@Min(0)
|
||||
private int zkSessionTimeoutMs = 30000;
|
||||
|
||||
@Config("druid.curator.compress")
|
||||
@Default("true")
|
||||
public abstract boolean enableCompression();
|
||||
@JsonProperty("compress")
|
||||
private boolean enableCompression = true;
|
||||
|
||||
public String getZkHosts()
|
||||
{
|
||||
return zkHosts;
|
||||
}
|
||||
|
||||
public void setZkHosts(String zkHosts)
|
||||
{
|
||||
this.zkHosts = zkHosts;
|
||||
}
|
||||
|
||||
public Integer getZkSessionTimeoutMs()
|
||||
{
|
||||
return zkSessionTimeoutMs;
|
||||
}
|
||||
|
||||
public void setZkSessionTimeoutMs(Integer zkSessionTimeoutMs)
|
||||
{
|
||||
this.zkSessionTimeoutMs = zkSessionTimeoutMs;
|
||||
}
|
||||
|
||||
public Boolean getEnableCompression()
|
||||
{
|
||||
return enableCompression;
|
||||
}
|
||||
|
||||
public void setEnableCompression(Boolean enableCompression)
|
||||
{
|
||||
this.enableCompression = enableCompression;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.google.inject.Provides;
|
|||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.guice.ConfigProvider;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.LazySingleton;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.curator.framework.CuratorFrameworkFactory;
|
||||
|
@ -41,7 +42,9 @@ public class CuratorModule implements Module
|
|||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
ConfigProvider.bind(binder, CuratorConfig.class);
|
||||
JsonConfigProvider.bind(
|
||||
binder, "druid.zk.service",
|
||||
CuratorConfig.class);
|
||||
}
|
||||
|
||||
@Provides @LazySingleton
|
||||
|
@ -52,7 +55,7 @@ public class CuratorModule implements Module
|
|||
.connectString(config.getZkHosts())
|
||||
.sessionTimeoutMs(config.getZkSessionTimeoutMs())
|
||||
.retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30))
|
||||
.compressionProvider(new PotentiallyGzippedCompressionProvider(config.enableCompression()))
|
||||
.compressionProvider(new PotentiallyGzippedCompressionProvider(config.getEnableCompression()))
|
||||
.build();
|
||||
|
||||
lifecycle.addHandler(
|
||||
|
|
|
@ -45,8 +45,7 @@ public class ServerModule implements DruidModule
|
|||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
ConfigProvider.bind(binder, ZkPathsConfig.class);
|
||||
|
||||
JsonConfigProvider.bind(binder, "druid.zk.paths", ZkPathsConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid", DruidNode.class, Self.class);
|
||||
}
|
||||
|
||||
|
|
|
@ -43,6 +43,7 @@ import io.druid.server.DruidNode;
|
|||
import io.druid.server.coordination.AbstractDataSegmentAnnouncer;
|
||||
import io.druid.server.coordination.DataSegmentAnnouncer;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.curator.framework.recipes.leader.LeaderLatch;
|
||||
|
@ -80,6 +81,9 @@ public class DruidClusterBridge
|
|||
private final BridgeZkCoordinator bridgeZkCoordinator;
|
||||
private final Announcer announcer;
|
||||
private final ServerInventoryView<Object> serverInventoryView;
|
||||
private final ZkPathsConfig zkPathsConfig;
|
||||
|
||||
private final DruidServerMetadata druidServerMetadata;
|
||||
|
||||
private final Map<DataSegment, Integer> segments = Maps.newHashMap();
|
||||
private final Object lock = new Object();
|
||||
|
@ -91,6 +95,8 @@ public class DruidClusterBridge
|
|||
public DruidClusterBridge(
|
||||
ObjectMapper jsonMapper,
|
||||
DruidClusterBridgeConfig config,
|
||||
ZkPathsConfig zkPathsConfig,
|
||||
DruidServerMetadata druidServerMetadata,
|
||||
ScheduledExecutorFactory scheduledExecutorFactory,
|
||||
@Self DruidNode self,
|
||||
CuratorFramework curator,
|
||||
|
@ -104,10 +110,12 @@ public class DruidClusterBridge
|
|||
this.jsonMapper = jsonMapper;
|
||||
this.config = config;
|
||||
this.bridgeZkCoordinator = bridgeZkCoordinator;
|
||||
this.zkPathsConfig = zkPathsConfig;
|
||||
this.announcer = announcer;
|
||||
this.serverInventoryView = serverInventoryView;
|
||||
this.curator = curator;
|
||||
this.leaderLatch = leaderLatch;
|
||||
this.druidServerMetadata = druidServerMetadata;
|
||||
|
||||
this.exec = scheduledExecutorFactory.create(1, "Coordinator-Exec--%d");
|
||||
this.self = self;
|
||||
|
@ -212,7 +220,7 @@ public class DruidClusterBridge
|
|||
private LeaderLatch createNewLeaderLatch()
|
||||
{
|
||||
final LeaderLatch newLeaderLatch = new LeaderLatch(
|
||||
curator, ZKPaths.makePath(config.getConnectorPath(), BRIDGE_OWNER_NODE), self.getHostAndPort()
|
||||
curator, ZKPaths.makePath(zkPathsConfig.getConnectorPath(), BRIDGE_OWNER_NODE), self.getHostAndPort()
|
||||
);
|
||||
|
||||
newLeaderLatch.addListener(
|
||||
|
@ -309,13 +317,13 @@ public class DruidClusterBridge
|
|||
self.getHostAndPort(),
|
||||
totalMaxSize,
|
||||
NODE_TYPE,
|
||||
config.getTier(),
|
||||
config.getPriority()
|
||||
druidServerMetadata.getTier(),
|
||||
druidServerMetadata.getPriority()
|
||||
);
|
||||
|
||||
try {
|
||||
final String path = ZKPaths.makePath(config.getAnnouncementsPath(), self.getHostAndPort());
|
||||
log.info("Updating [%s] to have a maxSize of[%,d] bytes", self.getHostAndPort(), totalMaxSize);
|
||||
final String path = ZKPaths.makePath(zkPathsConfig.getAnnouncementsPath(), self.getHostAndPort());
|
||||
log.info("Updating [%s] to have a maxSize of[%,d] bytes", self.getHost(), totalMaxSize);
|
||||
announcer.update(path, jsonMapper.writeValueAsBytes(me));
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
|
|
@ -19,34 +19,50 @@
|
|||
|
||||
package io.druid.server.bridge;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import io.druid.client.DruidServerConfig;
|
||||
import org.joda.time.Duration;
|
||||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class DruidClusterBridgeConfig extends ZkPathsConfig
|
||||
public abstract class DruidClusterBridgeConfig
|
||||
{
|
||||
@Config("druid.server.tier")
|
||||
@Default(DruidServer.DEFAULT_TIER)
|
||||
public abstract String getTier();
|
||||
@JsonProperty
|
||||
private Duration startDelay = new Duration("PT300s");
|
||||
@JsonProperty
|
||||
private Duration period = new Duration("PT60s");
|
||||
@JsonProperty
|
||||
private String brokerServiceName = "broker";
|
||||
|
||||
@Config("druid.bridge.startDelay")
|
||||
@Default("PT300s")
|
||||
public abstract Duration getStartDelay();
|
||||
|
||||
@Config("druid.bridge.period")
|
||||
@Default("PT60s")
|
||||
public abstract Duration getPeriod();
|
||||
|
||||
@Config("druid.bridge.broker.serviceName")
|
||||
public abstract String getBrokerServiceName();
|
||||
|
||||
@Config("druid.server.priority")
|
||||
public int getPriority()
|
||||
public Duration getStartDelay()
|
||||
{
|
||||
return DruidServer.DEFAULT_PRIORITY;
|
||||
return startDelay;
|
||||
}
|
||||
|
||||
public void setStartDelay(Duration startDelay)
|
||||
{
|
||||
this.startDelay = startDelay;
|
||||
}
|
||||
|
||||
public Duration getPeriod()
|
||||
{
|
||||
return period;
|
||||
}
|
||||
|
||||
public void setPeriod(Duration period)
|
||||
{
|
||||
this.period = period;
|
||||
}
|
||||
|
||||
public String getBrokerServiceName()
|
||||
{
|
||||
return brokerServiceName;
|
||||
}
|
||||
|
||||
public void setBrokerServiceName(String brokerServiceName)
|
||||
{
|
||||
this.brokerServiceName = brokerServiceName;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,85 +19,105 @@
|
|||
|
||||
package io.druid.server.initialization;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.curator.utils.ZKPaths;
|
||||
import org.skife.config.Config;
|
||||
|
||||
public abstract class ZkPathsConfig
|
||||
public class ZkPathsConfig
|
||||
{
|
||||
@Config("druid.zk.paths.base")
|
||||
public String getZkBasePath()
|
||||
@JsonProperty
|
||||
private
|
||||
String base = "druid";
|
||||
@JsonProperty
|
||||
private
|
||||
String propertiesPath;
|
||||
@JsonProperty
|
||||
private
|
||||
String announcementsPath;
|
||||
@JsonProperty
|
||||
private
|
||||
String servedSegmentsPath;
|
||||
@JsonProperty
|
||||
private
|
||||
String liveSegmentsPath;
|
||||
@JsonProperty
|
||||
private
|
||||
String coordinatorPath;
|
||||
@JsonProperty
|
||||
private
|
||||
String loadQueuePath;
|
||||
@JsonProperty
|
||||
private
|
||||
String connectorPath;
|
||||
|
||||
public String getBase()
|
||||
{
|
||||
return "druid";
|
||||
return base;
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.propertiesPath")
|
||||
public String getPropertiesPath()
|
||||
{
|
||||
return defaultPath("properties");
|
||||
return (null == propertiesPath) ? defaultPath("properties") : propertiesPath;
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.announcementsPath")
|
||||
public String getAnnouncementsPath()
|
||||
{
|
||||
return defaultPath("announcements");
|
||||
return (null == announcementsPath) ? defaultPath("announcements") : announcementsPath;
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.servedSegmentsPath")
|
||||
public String getServedSegmentsPath()
|
||||
{
|
||||
return defaultPath("servedSegments");
|
||||
return (null == servedSegmentsPath) ? defaultPath("servedSegments") : servedSegmentsPath;
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.liveSegmentsPath")
|
||||
public String getLiveSegmentsPath()
|
||||
{
|
||||
return defaultPath("segments");
|
||||
return (null == liveSegmentsPath) ? defaultPath("segments") : liveSegmentsPath;
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.loadQueuePath")
|
||||
public String getLoadQueuePath()
|
||||
{
|
||||
return defaultPath("loadQueue");
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.coordinatorPath")
|
||||
public String getCoordinatorPath()
|
||||
{
|
||||
return defaultPath("coordinator");
|
||||
return (null == coordinatorPath) ? defaultPath("coordinator") : coordinatorPath;
|
||||
}
|
||||
|
||||
public String getLoadQueuePath()
|
||||
{
|
||||
return (null == loadQueuePath) ? defaultPath("loadQueue") : loadQueuePath;
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.connectorPath")
|
||||
public String getConnectorPath()
|
||||
{
|
||||
return defaultPath("connector");
|
||||
return (null == connectorPath) ? defaultPath("connector") : connectorPath;
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.indexer.announcementsPath")
|
||||
public String getIndexerAnnouncementPath()
|
||||
protected String defaultPath(final String subPath)
|
||||
{
|
||||
return defaultPath("indexer/announcements");
|
||||
return ZKPaths.makePath(getBase(), subPath);
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.indexer.tasksPath")
|
||||
public String getIndexerTaskPath()
|
||||
{
|
||||
return defaultPath("indexer/tasks");
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.indexer.statusPath")
|
||||
public String getIndexerStatusPath()
|
||||
{
|
||||
return defaultPath("indexer/status");
|
||||
}
|
||||
|
||||
@Config("druid.zk.paths.indexer.leaderLatchPath")
|
||||
public String getIndexerLeaderLatchPath()
|
||||
{
|
||||
return defaultPath("indexer/leaderLatchPath");
|
||||
}
|
||||
|
||||
private String defaultPath(final String subPath)
|
||||
{
|
||||
return ZKPaths.makePath(getZkBasePath(), subPath);
|
||||
@Override
|
||||
public boolean equals(Object other){
|
||||
if(null == other){
|
||||
return false;
|
||||
}
|
||||
if(this == other){
|
||||
return true;
|
||||
}
|
||||
if(!(other instanceof ZkPathsConfig)){
|
||||
return false;
|
||||
}
|
||||
ZkPathsConfig otherConfig = (ZkPathsConfig) other;
|
||||
if(
|
||||
this.getBase().equals(otherConfig.getBase()) &&
|
||||
this.getAnnouncementsPath().equals(otherConfig.getAnnouncementsPath()) &&
|
||||
this.getConnectorPath().equals(otherConfig.getConnectorPath()) &&
|
||||
this.getLiveSegmentsPath().equals(otherConfig.getLiveSegmentsPath()) &&
|
||||
this.getCoordinatorPath().equals(otherConfig.getCoordinatorPath()) &&
|
||||
this.getLoadQueuePath().equals(otherConfig.getLoadQueuePath()) &&
|
||||
this.getPropertiesPath().equals(otherConfig.getPropertiesPath()) &&
|
||||
this.getServedSegmentsPath().equals(otherConfig.getServedSegmentsPath())
|
||||
){
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -120,7 +120,7 @@ public class BatchServerInventoryViewTest
|
|||
new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return testBasePath;
|
||||
}
|
||||
|
@ -139,7 +139,7 @@ public class BatchServerInventoryViewTest
|
|||
new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return testBasePath;
|
||||
}
|
||||
|
@ -155,7 +155,7 @@ public class BatchServerInventoryViewTest
|
|||
new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return testBasePath;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013, 2014 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.curator;
|
||||
|
||||
import io.druid.guice.JsonConfigTesterBase;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
|
||||
public class CuratorConfigTest extends JsonConfigTesterBase<CuratorConfig>
|
||||
{
|
||||
@Test
|
||||
public void testHostName() throws IllegalAccessException, NoSuchMethodException, InvocationTargetException
|
||||
{
|
||||
propertyValues.put(getPropertyKey("host"),"fooHost");
|
||||
testProperties.putAll(propertyValues);
|
||||
configProvider.inject(testProperties, configurator);
|
||||
CuratorConfig config = configProvider.get().get();
|
||||
Assert.assertEquals("fooHost", config.getZkHosts());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,155 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013, 2014 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.guice;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.name.Names;
|
||||
import io.druid.initialization.Initialization;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.lang.reflect.Field;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.lang.reflect.ParameterizedType;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public abstract class JsonConfigTesterBase<T>
|
||||
{
|
||||
|
||||
protected static final String configPrefix = "druid.test.prefix";
|
||||
protected Injector injector;
|
||||
protected final Class<T> clazz = (Class<T>) ((ParameterizedType) getClass().getGenericSuperclass()).getActualTypeArguments()[0];
|
||||
|
||||
protected Map<String, String> propertyValues = new HashMap<>();
|
||||
protected int assertions = 0;
|
||||
protected Properties testProperties = new Properties();
|
||||
|
||||
protected static String getPropertyKey(String fieldName){
|
||||
return String.format(
|
||||
"%s.%s",
|
||||
configPrefix, fieldName
|
||||
);
|
||||
}
|
||||
protected static String getPropertyKey(Field field)
|
||||
{
|
||||
JsonProperty jsonProperty = field.getAnnotation(JsonProperty.class);
|
||||
if (null != jsonProperty) {
|
||||
return getPropertyKey(
|
||||
(jsonProperty.value() == null || jsonProperty.value().isEmpty())
|
||||
? field.getName()
|
||||
: jsonProperty.value()
|
||||
);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private final Module simpleJsonConfigModule = new Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/test");
|
||||
binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0);
|
||||
JsonConfigProvider.bind(binder, configPrefix, clazz);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
protected final void validateEntries(T config)
|
||||
throws IllegalAccessException, NoSuchMethodException, InvocationTargetException
|
||||
{
|
||||
for (Field field : clazz.getDeclaredFields()) {
|
||||
final String propertyKey = getPropertyKey(field);
|
||||
if (null != propertyKey) {
|
||||
field.setAccessible(true);
|
||||
String getter = String.format(
|
||||
"get%s%s",
|
||||
field.getName().substring(0, 1).toUpperCase(),
|
||||
field.getName().substring(1)
|
||||
);
|
||||
Method method = clazz.getDeclaredMethod(getter);
|
||||
final String value;
|
||||
if (null != method) {
|
||||
value = method.invoke(config).toString();
|
||||
} else {
|
||||
value = field.get(config).toString();
|
||||
}
|
||||
|
||||
Assert.assertEquals(propertyValues.get(propertyKey), value);
|
||||
++assertions;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected JsonConfigurator configurator;
|
||||
protected JsonConfigProvider<T> configProvider;
|
||||
|
||||
@Before
|
||||
public void setup() throws IllegalAccessException
|
||||
{
|
||||
assertions = 0;
|
||||
T fakeValues = EasyMock.createNiceMock(clazz);
|
||||
propertyValues.clear();
|
||||
testProperties.clear();
|
||||
for (Field field : clazz.getDeclaredFields()) {
|
||||
final String propertyKey = getPropertyKey(field);
|
||||
if (null != propertyKey) {
|
||||
field.setAccessible(true);
|
||||
if (field.getType().isAssignableFrom(String.class)) {
|
||||
propertyValues.put(propertyKey, UUID.randomUUID().toString());
|
||||
} else {
|
||||
propertyValues.put(propertyKey, field.get(fakeValues).toString());
|
||||
}
|
||||
}
|
||||
}
|
||||
testProperties.putAll(System.getProperties());
|
||||
testProperties.putAll(propertyValues);
|
||||
injector = Initialization.makeInjectorWithModules(
|
||||
GuiceInjectors.makeStartupInjector(),
|
||||
ImmutableList.<Object>of(simpleJsonConfigModule)
|
||||
);
|
||||
configurator = injector.getBinding(JsonConfigurator.class).getProvider().get();
|
||||
configProvider = JsonConfigProvider.of(configPrefix, clazz);
|
||||
}
|
||||
|
||||
@Test
|
||||
public final void simpleInjectionTest()
|
||||
throws IllegalAccessException, NoSuchMethodException, InvocationTargetException
|
||||
{
|
||||
configProvider.inject(testProperties, configurator);
|
||||
validateEntries(configProvider.get().get());
|
||||
Assert.assertEquals(propertyValues.size(), assertions);
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013, 2014 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.initialization;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.inject.Key;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.JsonConfigTesterBase;
|
||||
import io.druid.guice.JsonConfigurator;
|
||||
import io.druid.guice.annotations.Json;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import org.apache.curator.utils.ZKPaths;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class ZkPathsConfigTest extends JsonConfigTesterBase<ZkPathsConfig>
|
||||
{
|
||||
@Test
|
||||
public void testOverrideBaseOnlyConfig()
|
||||
throws IllegalAccessException, NoSuchMethodException, InvocationTargetException, IOException
|
||||
{
|
||||
JsonConfigurator configurator = injector.getBinding(JsonConfigurator.class).getProvider().get();
|
||||
|
||||
JsonConfigProvider<ZkPathsConfig> zkPathsConfig = JsonConfigProvider.of(configPrefix, ZkPathsConfig.class);
|
||||
testProperties.clear();
|
||||
String base = UUID.randomUUID().toString();
|
||||
testProperties.put(String.format("%s.base", configPrefix), base);
|
||||
zkPathsConfig.inject(testProperties, configurator);
|
||||
|
||||
propertyValues.clear();
|
||||
propertyValues.put(String.format("%s.base", configPrefix), base);
|
||||
propertyValues.put(String.format("%s.propertiesPath", configPrefix), ZKPaths.makePath(base, "properties"));
|
||||
propertyValues.put(String.format("%s.announcementsPath", configPrefix), ZKPaths.makePath(base, "announcements"));
|
||||
propertyValues.put(String.format("%s.servedSegmentsPath", configPrefix), ZKPaths.makePath(base, "servedSegments"));
|
||||
propertyValues.put(String.format("%s.liveSegmentsPath", configPrefix), ZKPaths.makePath(base, "segments"));
|
||||
propertyValues.put(String.format("%s.coordinatorPath", configPrefix), ZKPaths.makePath(base, "coordinator"));
|
||||
propertyValues.put(String.format("%s.loadQueuePath", configPrefix), ZKPaths.makePath(base, "loadQueue"));
|
||||
propertyValues.put(String.format("%s.connectorPath", configPrefix), ZKPaths.makePath(base, "connector"));
|
||||
|
||||
ZkPathsConfig zkPathsConfigObj = zkPathsConfig.get().get();
|
||||
validateEntries(zkPathsConfigObj);
|
||||
Assert.assertEquals(propertyValues.size(), assertions);
|
||||
|
||||
ObjectMapper jsonMapper = injector.getProvider(Key.get(ObjectMapper.class, Json.class)).get();
|
||||
String jsonVersion = jsonMapper.writeValueAsString(zkPathsConfigObj);
|
||||
|
||||
ZkPathsConfig zkPathsConfigObjDeSer = jsonMapper.readValue(jsonVersion, ZkPathsConfig.class);
|
||||
|
||||
Assert.assertEquals(zkPathsConfigObj, zkPathsConfigObjDeSer);
|
||||
}
|
||||
}
|
|
@ -90,12 +90,6 @@ public class DruidClusterBridgeTest
|
|||
ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
DruidClusterBridgeConfig config = new DruidClusterBridgeConfig()
|
||||
{
|
||||
@Override
|
||||
public String getTier()
|
||||
{
|
||||
return DruidServer.DEFAULT_TIER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Duration getStartDelay()
|
||||
{
|
||||
|
@ -107,18 +101,6 @@ public class DruidClusterBridgeTest
|
|||
{
|
||||
return new Duration(Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getBrokerServiceName()
|
||||
{
|
||||
return "testz0rz";
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPriority()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
ScheduledExecutorFactory factory = ScheduledExecutors.createFactory(new Lifecycle());
|
||||
|
@ -134,7 +116,7 @@ public class DruidClusterBridgeTest
|
|||
ZkPathsConfig zkPathsConfig = new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return "/druid";
|
||||
}
|
||||
|
@ -195,6 +177,8 @@ public class DruidClusterBridgeTest
|
|||
DruidClusterBridge bridge = new DruidClusterBridge(
|
||||
jsonMapper,
|
||||
config,
|
||||
zkPathsConfig,
|
||||
metadata,
|
||||
factory,
|
||||
me,
|
||||
localCf,
|
||||
|
|
|
@ -101,7 +101,7 @@ public class ZkCoordinatorTest extends CuratorTestBase
|
|||
final ZkPathsConfig zkPaths = new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return "/druid";
|
||||
}
|
||||
|
|
|
@ -107,7 +107,7 @@ public class BatchDataSegmentAnnouncerTest
|
|||
new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return testBasePath;
|
||||
}
|
||||
|
|
|
@ -99,7 +99,7 @@ public class DruidCoordinatorTest
|
|||
{
|
||||
|
||||
@Override
|
||||
public String getZkBasePath()
|
||||
public String getBase()
|
||||
{
|
||||
return "";
|
||||
}
|
||||
|
|
|
@ -109,7 +109,7 @@ public class CliBridge extends ServerRunnable
|
|||
.retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30))
|
||||
.compressionProvider(
|
||||
new PotentiallyGzippedCompressionProvider(
|
||||
bridgeCuratorConfig.enableCompression()
|
||||
bridgeCuratorConfig.getEnableCompression()
|
||||
)
|
||||
)
|
||||
.build();
|
||||
|
|
Loading…
Reference in New Issue