Merge pull request #131 from metamx/curator

Remove references to i0tec.zkClient and replace with Curator
This commit is contained in:
cheddar 2013-04-30 09:02:47 -07:00
commit cd535fcd79
109 changed files with 3297 additions and 4037 deletions

View File

@ -18,7 +18,8 @@
~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<groupId>com.metamx.druid</groupId>
<artifactId>druid-client</artifactId>
@ -73,15 +74,15 @@
<artifactId>config-magic</artifactId>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-client</artifactId>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-framework</artifactId>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-x-discovery</artifactId>
</dependency>
<dependency>
@ -166,14 +167,18 @@
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</dependency>
<dependency>
<groupId>com.github.sgroschupf</groupId>
<artifactId>zkclient</artifactId>
</dependency>
<dependency>
<groupId>com.google.code.simple-spring-memcached</groupId>
<artifactId>spymemcached</artifactId>
</dependency>
<dependency>
<groupId>org.antlr</groupId>
<artifactId>antlr4-runtime</artifactId>
</dependency>
<dependency>
<groupId>commons-cli</groupId>
<artifactId>commons-cli</artifactId>
</dependency>
<!-- Tests -->
<dependency>
@ -192,13 +197,11 @@
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.antlr</groupId>
<artifactId>antlr4-runtime</artifactId>
</dependency>
<dependency>
<groupId>commons-cli</groupId>
<artifactId>commons-cli</artifactId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-test</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
@ -215,16 +218,16 @@
</plugin>
<plugin>
<groupId>org.antlr</groupId>
<artifactId>antlr4-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>antlr4</goal>
</goals>
</execution>
</executions>
<groupId>org.antlr</groupId>
<artifactId>antlr4-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>antlr4</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</plugins>
</build>
</project>

View File

@ -25,6 +25,7 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.concurrent.ScheduledExecutors;
@ -32,10 +33,19 @@ import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DruidServerConfig;
import com.metamx.druid.client.ServerInventoryThingieConfig;
import com.metamx.druid.client.ServerInventoryView;
import com.metamx.druid.concurrent.Execs;
import com.metamx.druid.coordination.CuratorDataSegmentAnnouncer;
import com.metamx.druid.coordination.DataSegmentAnnouncer;
import com.metamx.druid.coordination.DruidServerMetadata;
import com.metamx.druid.curator.announcement.Announcer;
import com.metamx.druid.http.RequestLogger;
import com.metamx.druid.initialization.CuratorConfig;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ZkClientConfig;
import com.metamx.druid.initialization.ZkPathsConfig;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters;
@ -48,8 +58,7 @@ import com.metamx.metrics.Monitor;
import com.metamx.metrics.MonitorScheduler;
import com.metamx.metrics.MonitorSchedulerConfig;
import com.metamx.metrics.SysMonitor;
import com.metamx.phonebook.PhoneBook;
import org.I0Itec.zkclient.ZkClient;
import org.apache.curator.framework.CuratorFramework;
import org.joda.time.Duration;
import org.mortbay.jetty.Server;
import org.skife.config.ConfigurationObjectFactory;
@ -59,6 +68,8 @@ import java.lang.reflect.Field;
import java.util.Arrays;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
/**
@ -72,18 +83,23 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
private final ObjectMapper smileMapper;
private final Properties props;
private final ConfigurationObjectFactory configFactory;
private final String nodeType;
private PhoneBook phoneBook = null;
private DruidServerMetadata druidServerMetadata = null;
private ServiceEmitter emitter = null;
private List<Monitor> monitors = null;
private Server server = null;
private ZkClient zkClient;
private ScheduledExecutorFactory scheduledExecutorFactory;
private RequestLogger requestLogger;
private CuratorFramework curator = null;
private DataSegmentAnnouncer announcer = null;
private ZkPathsConfig zkPaths = null;
private ScheduledExecutorFactory scheduledExecutorFactory = null;
private RequestLogger requestLogger = null;
private ServerInventoryView serverInventoryView = null;
private boolean initialized = false;
public QueryableNode(
String nodeType,
Logger log,
Properties props,
Lifecycle lifecycle,
@ -108,19 +124,26 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
Preconditions.checkNotNull(configFactory, "configFactory");
Preconditions.checkState(smileMapper.getJsonFactory() instanceof SmileFactory, "smileMapper should use smile.");
this.nodeType = nodeType;
}
@SuppressWarnings("unchecked")
public T setZkClient(ZkClient zkClient)
public T setDruidServerMetadata(DruidServerMetadata druidServerMetadata)
{
checkFieldNotSetAndSet("zkClient", zkClient);
checkFieldNotSetAndSet("druidServerMetadata", druidServerMetadata);
return (T) this;
}
@SuppressWarnings("unchecked")
public T setPhoneBook(PhoneBook phoneBook)
public T setCuratorFramework(CuratorFramework curator)
{
checkFieldNotSetAndSet("phoneBook", phoneBook);
checkFieldNotSetAndSet("curator", curator);
return (T) this;
}
@SuppressWarnings("unchecked")
public T setAnnouncer(DataSegmentAnnouncer announcer)
{
checkFieldNotSetAndSet("announcer", announcer);
return (T) this;
}
@ -145,6 +168,13 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
return (T) this;
}
@SuppressWarnings("unchecked")
public T setZkPaths(ZkPathsConfig zkPaths)
{
checkFieldNotSetAndSet("zkPaths", zkPaths);
return (T) this;
}
@SuppressWarnings("unchecked")
public T setScheduledExecutorFactory(ScheduledExecutorFactory factory)
{
@ -159,6 +189,13 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
return (T) this;
}
@SuppressWarnings("unchecked")
public T setServerInventoryView(ServerInventoryView serverInventoryView)
{
checkFieldNotSetAndSet("serverInventoryView", serverInventoryView);
return (T) this;
}
@SuppressWarnings("unchecked")
public T registerJacksonSubtype(Class<?>... clazzes)
{
@ -200,16 +237,22 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
return configFactory;
}
public ZkClient getZkClient()
public DruidServerMetadata getDruidServerMetadata()
{
initializeZkClient();
return zkClient;
initializeDruidServerMetadata();
return druidServerMetadata;
}
public PhoneBook getPhoneBook()
public CuratorFramework getCuratorFramework()
{
initializePhoneBook();
return phoneBook;
initializeCuratorFramework();
return curator;
}
public DataSegmentAnnouncer getAnnouncer()
{
initializeAnnouncer();
return announcer;
}
public ServiceEmitter getEmitter()
@ -230,6 +273,12 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
return server;
}
public ZkPathsConfig getZkPaths()
{
initializeZkPaths();
return zkPaths;
}
public ScheduledExecutorFactory getScheduledExecutorFactory()
{
initializeScheduledExecutorFactory();
@ -242,6 +291,47 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
return requestLogger;
}
public ServerInventoryView getServerInventoryView()
{
initializeServerInventoryThingie();
return serverInventoryView;
}
private void initializeDruidServerMetadata()
{
if (druidServerMetadata == null) {
final DruidServerConfig serverConfig = getConfigFactory().build(DruidServerConfig.class);
setDruidServerMetadata(
new DruidServerMetadata(
serverConfig.getServerName(),
serverConfig.getHost(),
serverConfig.getMaxSize(),
nodeType,
serverConfig.getTier()
)
);
}
}
private void initializeServerInventoryThingie()
{
if (serverInventoryView == null) {
final ExecutorService exec = Executors.newFixedThreadPool(
1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build()
);
setServerInventoryView(
new ServerInventoryView(
getConfigFactory().build(ServerInventoryThingieConfig.class),
getZkPaths(),
getCuratorFramework(),
exec,
getJsonMapper()
)
);
lifecycle.addManagedInstance(serverInventoryView);
}
}
private void initializeRequestLogger()
{
if (requestLogger == null) {
@ -268,6 +358,13 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
}
}
private void initializeZkPaths()
{
if (zkPaths == null) {
setZkPaths(getConfigFactory().build(ZkPathsConfig.class));
}
}
private void initializeScheduledExecutorFactory()
{
if (scheduledExecutorFactory == null) {
@ -275,24 +372,26 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
}
}
private void initializeZkClient()
private void initializeCuratorFramework()
{
if (zkClient == null) {
setZkClient(Initialization.makeZkClient(configFactory.build(ZkClientConfig.class), lifecycle));
if (curator == null) {
try {
setCuratorFramework(Initialization.makeCuratorFramework(configFactory.build(CuratorConfig.class), lifecycle));
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
}
private void initializePhoneBook()
private void initializeAnnouncer()
{
if (phoneBook == null) {
setPhoneBook(
Initialization.createPhoneBook(
jsonMapper,
getZkClient(),
"PhoneBook--%s",
lifecycle
)
);
if (announcer == null) {
final Announcer announcer = new Announcer(getCuratorFramework(), Execs.singleThreaded("Announcer-%s"));
lifecycle.addManagedInstance(announcer);
setAnnouncer(new CuratorDataSegmentAnnouncer(getDruidServerMetadata(), getZkPaths(), announcer, getJsonMapper()));
lifecycle.addManagedInstance(getAnnouncer(), Lifecycle.Stage.LAST);
}
}
@ -343,8 +442,7 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
private void initializeEmitter()
{
if (emitter == null) {
final HttpClientConfig.Builder configBuilder = HttpClientConfig.builder()
.withNumConnections(1);
final HttpClientConfig.Builder configBuilder = HttpClientConfig.builder().withNumConnections(1);
final String emitterTimeoutDuration = props.getProperty("druid.emitter.timeOut");
if (emitterTimeoutDuration != null) {

View File

@ -20,25 +20,25 @@
package com.metamx.druid.client;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.metamx.common.logger.Logger;
import com.metamx.druid.VersionedIntervalTimeline;
import com.metamx.druid.client.selector.ServerSelector;
import com.metamx.druid.partition.PartitionChunk;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.query.QueryToolChestWarehouse;
import com.metamx.http.client.HttpClient;
import javax.annotation.Nullable;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
/**
*/
public class BrokerServerView implements MutableServerView
public class BrokerServerView implements TimelineServerView
{
private static final Logger log = new Logger(BrokerServerView.class);
@ -47,38 +47,69 @@ public class BrokerServerView implements MutableServerView
private final ConcurrentMap<DruidServer, DirectDruidClient> clients;
private final Map<String, ServerSelector> selectors;
private final Map<String, VersionedIntervalTimeline<String, ServerSelector>> timelines;
private final ConcurrentMap<ServerCallback, Executor> serverCallbacks;
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks;
private final QueryToolChestWarehouse warehose;
private final ObjectMapper smileMapper;
private final HttpClient httpClient;
private final ServerView baseView;
public BrokerServerView(
QueryToolChestWarehouse warehose,
ObjectMapper smileMapper,
HttpClient httpClient
HttpClient httpClient,
ServerView baseView,
ExecutorService exec
)
{
this.warehose = warehose;
this.smileMapper = smileMapper;
this.httpClient = httpClient;
this.baseView = baseView;
this.clients = Maps.newConcurrentMap();
this.selectors = Maps.newHashMap();
this.timelines = Maps.newHashMap();
this.serverCallbacks = Maps.newConcurrentMap();
this.segmentCallbacks = Maps.newConcurrentMap();
baseView.registerSegmentCallback(
exec,
new ServerView.SegmentCallback()
{
@Override
public ServerView.CallbackAction segmentAdded(DruidServer server, DataSegment segment)
{
serverAddedSegment(server, segment);
return ServerView.CallbackAction.CONTINUE;
}
@Override
public ServerView.CallbackAction segmentRemoved(final DruidServer server, DataSegment segment)
{
serverRemovedSegment(server, segment);
return ServerView.CallbackAction.CONTINUE;
}
}
);
baseView.registerServerCallback(
exec,
new ServerView.ServerCallback()
{
@Override
public ServerView.CallbackAction serverRemoved(DruidServer server)
{
removeServer(server);
return ServerView.CallbackAction.CONTINUE;
}
}
);
}
@Override
public void clear()
{
synchronized (lock) {
final Iterator<DruidServer> clientsIter = clients.keySet().iterator();
while (clientsIter.hasNext()) {
DruidServer server = clientsIter.next();
clientsIter.remove();
runServerCallbacks(server);
}
timelines.clear();
@ -89,47 +120,38 @@ public class BrokerServerView implements MutableServerView
selectorsIter.remove();
while (!selector.isEmpty()) {
final DruidServer pick = selector.pick();
runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(@Nullable SegmentCallback input)
{
return input.segmentRemoved(pick, selector.getSegment());
}
}
);
selector.removeServer(pick);
}
}
}
}
@Override
public void addServer(DruidServer server)
private void addServer(DruidServer server)
{
QueryRunner exists = clients.put(server, new DirectDruidClient(warehose, smileMapper, httpClient, server.getHost()));
QueryRunner exists = clients.put(server, makeDirectClient(server));
if (exists != null) {
log.warn("QueryRunner for server[%s] already existed!?", server);
}
}
@Override
public void removeServer(DruidServer server)
private DirectDruidClient makeDirectClient(DruidServer server)
{
return new DirectDruidClient(warehose, smileMapper, httpClient, server.getHost());
}
private void removeServer(DruidServer server)
{
clients.remove(server);
for (DataSegment segment : server.getSegments().values()) {
serverRemovedSegment(server, segment.getIdentifier());
serverRemovedSegment(server, segment);
}
runServerCallbacks(server);
}
@Override
public void serverAddedSegment(final DruidServer server, final DataSegment segment)
private void serverAddedSegment(final DruidServer server, final DataSegment segment)
{
String segmentId = segment.getIdentifier();
synchronized (lock) {
log.info("Adding segment[%s] for server[%s]", segment, server);
log.debug("Adding segment[%s] for server[%s]", segment, server);
ServerSelector selector = selectors.get(segmentId);
if (selector == null) {
@ -145,28 +167,20 @@ public class BrokerServerView implements MutableServerView
selectors.put(segmentId, selector);
}
if (!clients.containsKey(server)) {
addServer(server);
}
selector.addServer(server);
runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(@Nullable SegmentCallback input)
{
return input.segmentAdded(server, segment);
}
}
);
}
}
@Override
public void serverRemovedSegment(final DruidServer server, final String segmentId)
private void serverRemovedSegment(DruidServer server, DataSegment segment)
{
String segmentId = segment.getIdentifier();
final ServerSelector selector;
synchronized (lock) {
log.info("Removing segment[%s] from server[%s].", segmentId, server);
log.debug("Removing segment[%s] from server[%s].", segmentId, server);
selector = selectors.get(segmentId);
if (selector == null) {
@ -183,35 +197,25 @@ public class BrokerServerView implements MutableServerView
}
if (selector.isEmpty()) {
DataSegment segment = selector.getSegment();
VersionedIntervalTimeline<String, ServerSelector> timeline = timelines.get(segment.getDataSource());
selectors.remove(segmentId);
if (timeline.remove(
segment.getInterval(),
segment.getVersion(),
segment.getShardSpec().createChunk(selector)
) == null) {
final PartitionChunk<ServerSelector> removedPartition = timeline.remove(
segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)
);
if (removedPartition == null) {
log.warn(
"Asked to remove timeline entry[interval: %s, version: %s] that doesn't exist",
segment.getInterval(),
segment.getVersion()
);
}
runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(@Nullable SegmentCallback input)
{
return input.segmentRemoved(server, selector.getSegment());
}
}
);
}
}
}
@Override
public VersionedIntervalTimeline<String, ServerSelector> getTimeline(String dataSource)
{
@ -231,50 +235,12 @@ public class BrokerServerView implements MutableServerView
@Override
public void registerServerCallback(Executor exec, ServerCallback callback)
{
serverCallbacks.put(callback, exec);
baseView.registerServerCallback(exec, callback);
}
@Override
public void registerSegmentCallback(Executor exec, SegmentCallback callback)
{
segmentCallbacks.put(callback, exec);
}
private void runSegmentCallbacks(
final Function<SegmentCallback, CallbackAction> fn
)
{
for (final Map.Entry<SegmentCallback, Executor> entry : segmentCallbacks.entrySet()) {
entry.getValue().execute(
new Runnable()
{
@Override
public void run()
{
if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) {
segmentCallbacks.remove(entry.getKey());
}
}
}
);
}
}
private void runServerCallbacks(final DruidServer server)
{
for (final Map.Entry<ServerCallback, Executor> entry : serverCallbacks.entrySet()) {
entry.getValue().execute(
new Runnable()
{
@Override
public void run()
{
if (CallbackAction.UNREGISTER == entry.getKey().serverRemoved(server)) {
serverCallbacks.remove(entry.getKey());
}
}
}
);
}
baseView.registerSegmentCallback(exec, callback);
}
}

View File

@ -76,13 +76,13 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
private static final Logger log = new Logger(CachingClusteredClient.class);
private final QueryToolChestWarehouse warehouse;
private final ServerView serverView;
private final TimelineServerView serverView;
private final Cache cache;
private final ObjectMapper objectMapper;
public CachingClusteredClient(
QueryToolChestWarehouse warehouse,
ServerView serverView,
TimelineServerView serverView,
Cache cache,
ObjectMapper objectMapper
)

View File

@ -1,163 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.client;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.Pair;
import com.metamx.common.logger.Logger;
import com.metamx.phonebook.PhoneBook;
import com.metamx.phonebook.PhoneBookPeon;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
/**
*/
public class ClientInventoryManager extends InventoryManager<DruidServer>
{
private static final Logger log = new Logger(ClientInventoryManager.class);
private final Object lock = new Object();
private final Executor exec;
private final MutableServerView serverView;
public ClientInventoryManager(
final InventoryManagerConfig config,
final PhoneBook yp,
final MutableServerView serverView
)
{
super(log, config, yp);
this.serverView = serverView;
this.exec = Executors.newFixedThreadPool(
1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("CIV-Execution-%d").build()
);
setStrategy(
new InventoryManagementStrategy<DruidServer>()
{
@Override
public Class<DruidServer> getContainerClass()
{
return DruidServer.class;
}
@Override
public Pair<String, PhoneBookPeon<?>> makeSubListener(final DruidServer server)
{
ClientInventoryManager.this.serverView.addServer(server);
return Pair.<String, PhoneBookPeon<?>>of(
server.getName(),
new PhoneBookPeon<DataSegment>()
{
@Override
public Class<DataSegment> getObjectClazz()
{
return DataSegment.class;
}
@Override
public void newEntry(String segmentId, DataSegment segment)
{
exec.execute(new AddSegmentRunnable(server, segment));
server.addDataSegment(segmentId, segment);
}
@Override
public void entryRemoved(String segmentId)
{
exec.execute(new RemoveSegmentRunnable(server, segmentId));
server.removeDataSegment(segmentId);
}
}
);
}
@Override
public void objectRemoved(DruidServer server)
{
ClientInventoryManager.this.serverView.removeServer(server);
}
@Override
public boolean doesSerde()
{
return false;
}
@Override
public DruidServer deserialize(String name, Map<String, String> properties)
{
throw new UnsupportedOperationException();
}
}
);
}
@Override
protected void doStop()
{
synchronized (lock) {
serverView.clear();
}
}
private class RemoveSegmentRunnable implements Runnable
{
private final DruidServer server;
private final String segmentId;
public RemoveSegmentRunnable(DruidServer server, String segmentId)
{
this.server = server;
this.segmentId = segmentId;
}
@Override
public void run()
{
serverView.serverRemovedSegment(server, segmentId);
}
}
private class AddSegmentRunnable implements Runnable
{
private final DruidServer server;
private final DataSegment segment;
public AddSegmentRunnable(DruidServer server, DataSegment segment)
{
this.server = server;
this.segment = segment;
}
@Override
public void run()
{
serverView.serverAddedSegment(server, segment);
}
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.logger.Logger;
import com.metamx.druid.coordination.DruidServerMetadata;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@ -36,14 +37,10 @@ public class DruidServer implements Comparable
private final Object lock = new Object();
private final String name;
private final ConcurrentMap<String, DruidDataSource> dataSources;
private final ConcurrentMap<String, DataSegment> segments;
private final String host;
private final long maxSize;
private final String type;
private final String tier;
private final DruidServerMetadata metadata;
private volatile long currSize;
@ -70,11 +67,7 @@ public class DruidServer implements Comparable
@JsonProperty("tier") String tier
)
{
this.name = name;
this.host = host;
this.maxSize = maxSize;
this.type = type;
this.tier = tier;
this.metadata = new DruidServerMetadata(name, host, maxSize, type, tier);
this.dataSources = new ConcurrentHashMap<String, DruidDataSource>();
this.segments = new ConcurrentHashMap<String, DataSegment>();
@ -82,24 +75,18 @@ public class DruidServer implements Comparable
public String getName()
{
return name;
return metadata.getName();
}
public Map<String, String> getStringProps()
public DruidServerMetadata getMetadata()
{
return ImmutableMap.of(
"name", name,
"host", host,
"maxSize", String.valueOf(maxSize),
"type", type,
"tier", tier
);
return metadata;
}
@JsonProperty
public String getHost()
{
return host;
return metadata.getHost();
}
@JsonProperty
@ -111,19 +98,19 @@ public class DruidServer implements Comparable
@JsonProperty
public long getMaxSize()
{
return maxSize;
return metadata.getMaxSize();
}
@JsonProperty
public String getType()
{
return type;
return metadata.getType();
}
@JsonProperty
public String getTier()
{
return tier;
return metadata.getTier();
}
@JsonProperty
@ -159,13 +146,23 @@ public class DruidServer implements Comparable
return this;
}
public DruidServer addDataSegments(DruidServer server)
{
synchronized (lock) {
for (Map.Entry<String, DataSegment> entry : server.segments.entrySet()) {
addDataSegment(entry.getKey(), entry.getValue());
}
}
return this;
}
public DruidServer removeDataSegment(String segmentName)
{
synchronized (lock) {
DataSegment segment = segments.get(segmentName);
if (segment == null) {
log.warn("Asked to remove data segment that doesn't exist!? server[%s], segment[%s]", name, segmentName);
log.warn("Asked to remove data segment that doesn't exist!? server[%s], segment[%s]", getName(), segmentName);
return this;
}
@ -176,7 +173,7 @@ public class DruidServer implements Comparable
"Asked to remove data segment from dataSource[%s] that doesn't exist, but the segment[%s] exists!?!?!?! wtf? server[%s]",
segment.getDataSource(),
segmentName,
name
getName()
);
return this;
}
@ -214,7 +211,7 @@ public class DruidServer implements Comparable
DruidServer that = (DruidServer) o;
if (name != null ? !name.equals(that.name) : that.name != null) {
if (getName() != null ? !getName().equals(that.getName()) : that.getName() != null) {
return false;
}
@ -224,19 +221,13 @@ public class DruidServer implements Comparable
@Override
public int hashCode()
{
return name != null ? name.hashCode() : 0;
return getName() != null ? getName().hashCode() : 0;
}
@Override
public String toString()
{
return "DruidServer{" +
"name='" + name + '\'' +
", host='" + host + '\'' +
", maxSize=" + maxSize +
", type=" + type +
", tier=" + tier +
'}';
return metadata.toString();
}
@Override
@ -249,6 +240,6 @@ public class DruidServer implements Comparable
return 1;
}
return name.compareTo(((DruidServer) o).name);
return getName().compareTo(((DruidServer) o).getName());
}
}

View File

@ -1,219 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.client;
import com.metamx.common.ISE;
import com.metamx.common.Pair;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.phonebook.PhoneBook;
import com.metamx.phonebook.PhoneBookPeon;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
/**
*/
public abstract class InventoryManager<T>
{
private final Object lock = new Object();
private volatile boolean started = false;
private final MasterPeon masterPeon;
private final ConcurrentHashMap<String, PhoneBookPeon<?>> dataSourcePeons;
private final ConcurrentHashMap<String, T> dataSources;
private final Logger log;
private final PhoneBook yp;
private final InventoryManagerConfig config;
private volatile InventoryManagementStrategy<T> strategy = null;
public InventoryManager(
Logger log,
InventoryManagerConfig config,
PhoneBook zkPhoneBook
)
{
this.log = log;
this.config = config;
this.yp = zkPhoneBook;
this.masterPeon = new MasterPeon();
this.dataSourcePeons = new ConcurrentHashMap<String, PhoneBookPeon<?>>();
this.dataSources = new ConcurrentHashMap<String, T>();
}
public InventoryManager(
Logger log,
InventoryManagerConfig config,
PhoneBook zkPhoneBook,
InventoryManagementStrategy<T> strategy
)
{
this(log, config, zkPhoneBook);
setStrategy(strategy);
}
public void setStrategy(InventoryManagementStrategy<T> strategy)
{
if (this.strategy != null) {
throw new ISE("Management can only handle a single strategy, you cannot change your strategy.");
}
this.strategy = strategy;
}
@LifecycleStart
public void start()
{
synchronized (lock) {
if (started) {
return;
}
if (strategy == null) {
throw new ISE("Management requires a strategy, please provide a strategy.");
}
if (!yp.isStarted()) {
throw new ISE("Management does not work without a running yellow pages.");
}
yp.registerListener(config.getInventoryIdPath(), masterPeon);
doStart();
started = true;
}
}
protected void doStart() {};
@LifecycleStop
public void stop()
{
synchronized (lock) {
if (!started) {
return;
}
yp.unregisterListener(config.getInventoryIdPath(), masterPeon);
for (Map.Entry<String, PhoneBookPeon<?>> entry : dataSourcePeons.entrySet()) {
yp.unregisterListener(
yp.combineParts(Arrays.asList(config.getInventoryPath(), entry.getKey())), entry.getValue()
);
}
dataSources.clear();
dataSourcePeons.clear();
doStop();
started = false;
}
}
protected void doStop() {};
public boolean isStarted()
{
return started;
}
public T getInventoryValue(String key)
{
return dataSources.get(key);
}
public Iterable<T> getInventory()
{
return dataSources.values();
}
public void remove(List<String> nodePath)
{
yp.unpost(config.getInventoryIdPath(), yp.combineParts(nodePath));
}
private class MasterPeon implements PhoneBookPeon
{
@Override
public Class getObjectClazz()
{
return strategy.doesSerde() ? Object.class : strategy.getContainerClass();
}
@Override
public void newEntry(final String name, Object baseObject)
{
synchronized (lock) {
if (!started) {
return;
}
log.info("New inventory container[%s]!", name);
if (strategy.doesSerde()) { // Hack to work around poor serialization choice
baseObject = strategy.deserialize(name, (Map<String, String>) baseObject);
}
Object shouldBeNull = dataSources.put(name, strategy.getContainerClass().cast(baseObject));
if (shouldBeNull != null) {
log.warn(
"Just put key[%s] into dataSources and what was there wasn't null!? It was[%s]", name, shouldBeNull
);
}
Pair<String, PhoneBookPeon<?>> pair = strategy.makeSubListener(strategy.getContainerClass().cast(baseObject));
shouldBeNull = dataSourcePeons.put(pair.lhs, pair.rhs);
if (shouldBeNull != null) {
log.warn(
"Just put key[%s] into dataSourcePeons and what was there wasn't null!? It was[%s]", name, shouldBeNull
);
}
String serviceName = yp.combineParts(Arrays.asList(config.getInventoryPath(), pair.lhs));
log.info("Putting watch on [%s]", serviceName);
yp.registerListener(serviceName, pair.rhs);
}
}
@Override
public void entryRemoved(String name)
{
synchronized (lock) {
if (!started) {
return;
}
log.info("Inventory container[%s] removed, deleting.", name);
T removed = dataSources.remove(name);
if (removed != null) {
strategy.objectRemoved(removed);
}
else {
log.warn("Removed empty element at path[%s]", name);
}
yp.unregisterListener(
yp.combineParts(Arrays.asList(config.getInventoryPath(), name)), dataSourcePeons.remove(name)
);
}
}
}
}

View File

@ -1,117 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.client;
import com.google.common.collect.Maps;
import com.metamx.common.logger.Logger;
import com.metamx.druid.VersionedIntervalTimeline;
import com.metamx.druid.client.selector.ServerSelector;
import com.metamx.druid.query.QueryRunner;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executor;
/**
*/
public class OnlyNewSegmentWatcherServerView implements MutableServerView
{
private static final Logger log = new Logger(OnlyNewSegmentWatcherServerView.class);
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks;
public OnlyNewSegmentWatcherServerView()
{
this.segmentCallbacks = Maps.newConcurrentMap();
}
@Override
public void clear()
{
}
@Override
public void addServer(DruidServer server)
{
}
@Override
public void removeServer(DruidServer server)
{
}
@Override
public void serverAddedSegment(DruidServer server, DataSegment segment)
{
runSegmentCallbacks(server, segment);
}
@Override
public void serverRemovedSegment(DruidServer server, String segmentId)
{
}
@Override
public VersionedIntervalTimeline<String, ServerSelector> getTimeline(String dataSource)
{
throw new UnsupportedOperationException();
}
@Override
public QueryRunner getQueryRunner(DruidServer server)
{
throw new UnsupportedOperationException();
}
@Override
public void registerServerCallback(Executor exec, ServerCallback callback)
{
throw new UnsupportedOperationException();
}
@Override
public void registerSegmentCallback(Executor exec, SegmentCallback callback)
{
segmentCallbacks.put(callback, exec);
}
private void runSegmentCallbacks(final DruidServer server, final DataSegment segment)
{
Iterator<Map.Entry<SegmentCallback, Executor>> iter = segmentCallbacks.entrySet().iterator();
while (iter.hasNext()) {
final Map.Entry<SegmentCallback, Executor> entry = iter.next();
entry.getValue().execute(
new Runnable()
{
@Override
public void run()
{
if (CallbackAction.UNREGISTER == entry.getKey().segmentAdded(server, segment)) {
segmentCallbacks.remove(entry.getKey());
}
}
}
);
}
}
}

View File

@ -1,144 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.client;
import com.metamx.common.Pair;
import com.metamx.common.logger.Logger;
import com.metamx.phonebook.PhoneBook;
import com.metamx.phonebook.PhoneBookPeon;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
/**
*/
public class ServerInventoryManager extends InventoryManager<DruidServer>
{
private static final Map<String, Integer> removedSegments = new ConcurrentHashMap<String, Integer>();
public ServerInventoryManager(
ServerInventoryManagerConfig config,
PhoneBook yp
)
{
super(
new Logger(ServerInventoryManager.class.getName() + "." + config.getServerInventoryPath()),
new InventoryManagerConfig(
config.getServerIdPath(),
config.getServerInventoryPath()
),
yp,
new ServerInventoryManagementStrategy(
new Logger(
ServerInventoryManager.class.getName() + "." + config.getServerInventoryPath()
),
config.getRemovedSegmentLifetime()
)
);
}
private static class ServerInventoryManagementStrategy implements InventoryManagementStrategy<DruidServer>
{
private final Logger log;
private final int segmentLifetime;
ServerInventoryManagementStrategy(Logger log, int segmentLifetime)
{
this.log = log;
this.segmentLifetime = segmentLifetime;
}
@Override
public Class<DruidServer> getContainerClass()
{
return DruidServer.class;
}
@Override
public Pair<String, PhoneBookPeon<?>> makeSubListener(final DruidServer druidServer)
{
return new Pair<String, PhoneBookPeon<?>>(
druidServer.getName(),
new PhoneBookPeon<DataSegment>()
{
@Override
public Class<DataSegment> getObjectClazz()
{
return DataSegment.class;
}
@Override
public void newEntry(String name, DataSegment segment)
{
log.info("Server[%s] added new DataSegment[%s]", druidServer.getName(), segment);
druidServer.addDataSegment(name, segment);
}
@Override
public void entryRemoved(String name)
{
log.info("Entry [%s] deleted", name);
removedSegments.put(druidServer.getSegment(name).getIdentifier(), segmentLifetime);
druidServer.removeDataSegment(name);
log.info("Server[%s] removed dataSegment[%s]", druidServer.getName(), name);
}
}
);
}
@Override
public void objectRemoved(DruidServer baseObject)
{
}
@Override
public boolean doesSerde()
{
return false;
}
@Override
public DruidServer deserialize(String name, Map<String, String> properties)
{
throw new UnsupportedOperationException();
}
}
public int lookupSegmentLifetime(DataSegment segment)
{
Integer lifetime = removedSegments.get(segment.getIdentifier());
return (lifetime == null) ? 0 : lifetime;
}
public void decrementRemovedSegmentsLifetime()
{
for (Iterator<Map.Entry<String, Integer>> mapIter = removedSegments.entrySet().iterator(); mapIter.hasNext(); ) {
Map.Entry<String, Integer> segment = mapIter.next();
int lifetime = segment.getValue() - 1;
if (lifetime < 0) {
mapIter.remove();
} else {
segment.setValue(lifetime);
}
}
}
}

View File

@ -24,14 +24,8 @@ import org.skife.config.Default;
/**
*/
public abstract class ServerInventoryManagerConfig
public abstract class ServerInventoryThingieConfig
{
@Config("druid.zk.paths.announcementsPath")
public abstract String getServerIdPath();
@Config("druid.zk.paths.servedSegmentsPath")
public abstract String getServerInventoryPath();
@Config("druid.master.removedSegmentLifetime")
@Default("1")
public abstract int getRemovedSegmentLifetime();

View File

@ -0,0 +1,298 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.client;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.MapMaker;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.curator.inventory.CuratorInventoryManager;
import com.metamx.druid.curator.inventory.CuratorInventoryManagerStrategy;
import com.metamx.druid.curator.inventory.InventoryManagerConfig;
import com.metamx.druid.initialization.ZkPathsConfig;
import com.metamx.emitter.EmittingLogger;
import org.apache.curator.framework.CuratorFramework;
import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
/**
*/
public class ServerInventoryView implements ServerView
{
private static final EmittingLogger log = new EmittingLogger(ServerInventoryView.class);
private final CuratorInventoryManager<DruidServer, DataSegment> inventoryManager;
private final AtomicBoolean started = new AtomicBoolean(false);
private final ConcurrentMap<ServerCallback, Executor> serverCallbacks = new MapMaker().makeMap();
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks = new MapMaker().makeMap();
private static final Map<String, Integer> removedSegments = new MapMaker().makeMap();
public ServerInventoryView(
final ServerInventoryThingieConfig config,
final ZkPathsConfig zkPaths,
final CuratorFramework curator,
final ExecutorService exec,
final ObjectMapper jsonMapper
)
{
inventoryManager = new CuratorInventoryManager<DruidServer, DataSegment>(
curator,
new InventoryManagerConfig()
{
@Override
public String getContainerPath()
{
return zkPaths.getAnnouncementsPath();
}
@Override
public String getInventoryPath()
{
return zkPaths.getServedSegmentsPath();
}
},
exec,
new CuratorInventoryManagerStrategy<DruidServer, DataSegment>()
{
@Override
public DruidServer deserializeContainer(byte[] bytes)
{
try {
return jsonMapper.readValue(bytes, DruidServer.class);
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
@Override
public byte[] serializeContainer(DruidServer container)
{
try {
return jsonMapper.writeValueAsBytes(container);
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
}
}
@Override
public DataSegment deserializeInventory(byte[] bytes)
{
try {
return jsonMapper.readValue(bytes, DataSegment.class);
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
@Override
public byte[] serializeInventory(DataSegment inventory)
{
try {
return jsonMapper.writeValueAsBytes(inventory);
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
}
}
@Override
public void newContainer(DruidServer container)
{
log.info("New Server[%s]", container);
}
@Override
public void deadContainer(DruidServer deadContainer)
{
log.info("Server Disdappeared[%s]", deadContainer);
runServerCallbacks(deadContainer);
}
@Override
public DruidServer updateContainer(DruidServer oldContainer, DruidServer newContainer)
{
return newContainer.addDataSegments(oldContainer);
}
@Override
public DruidServer addInventory(final DruidServer container, String inventoryKey, final DataSegment inventory)
{
log.info("Server[%s] added segment[%s]", container.getName(), inventory);
final DruidServer retVal = container.addDataSegment(inventoryKey, inventory);
runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(SegmentCallback input)
{
return input.segmentAdded(container, inventory);
}
}
);
return retVal;
}
@Override
public DruidServer removeInventory(final DruidServer container, String inventoryKey)
{
log.info("Server[%s] removed segment[%s]", container.getName(), inventoryKey);
final DataSegment segment = container.getSegment(inventoryKey);
final DruidServer retVal = container.removeDataSegment(inventoryKey);
runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(SegmentCallback input)
{
return input.segmentRemoved(container, segment);
}
}
);
removedSegments.put(inventoryKey, config.getRemovedSegmentLifetime());
return retVal;
}
}
);
}
public int lookupSegmentLifetime(DataSegment segment)
{
Integer lifetime = removedSegments.get(segment.getIdentifier());
return (lifetime == null) ? 0 : lifetime;
}
public void decrementRemovedSegmentsLifetime()
{
for (Iterator<Map.Entry<String, Integer>> mapIter = removedSegments.entrySet().iterator(); mapIter.hasNext(); ) {
Map.Entry<String, Integer> segment = mapIter.next();
int lifetime = segment.getValue() - 1;
if (lifetime < 0) {
mapIter.remove();
} else {
segment.setValue(lifetime);
}
}
}
@LifecycleStart
public void start() throws Exception
{
synchronized (started) {
if (!started.get()) {
inventoryManager.start();
started.set(true);
}
}
}
@LifecycleStop
public void stop() throws IOException
{
synchronized (started) {
if (started.getAndSet(false)) {
inventoryManager.stop();
}
}
}
public boolean isStarted()
{
return started.get();
}
public DruidServer getInventoryValue(String containerKey)
{
return inventoryManager.getInventoryValue(containerKey);
}
public Iterable<DruidServer> getInventory()
{
return inventoryManager.getInventory();
}
@Override
public void registerServerCallback(Executor exec, ServerCallback callback)
{
serverCallbacks.put(callback, exec);
}
@Override
public void registerSegmentCallback(Executor exec, SegmentCallback callback)
{
segmentCallbacks.put(callback, exec);
}
private void runSegmentCallbacks(
final Function<SegmentCallback, CallbackAction> fn
)
{
for (final Map.Entry<SegmentCallback, Executor> entry : segmentCallbacks.entrySet()) {
entry.getValue().execute(
new Runnable()
{
@Override
public void run()
{
if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) {
segmentCallbacks.remove(entry.getKey());
}
}
}
);
}
}
private void runServerCallbacks(final DruidServer server)
{
for (final Map.Entry<ServerCallback, Executor> entry : serverCallbacks.entrySet()) {
entry.getValue().execute(
new Runnable()
{
@Override
public void run()
{
if (CallbackAction.UNREGISTER == entry.getKey().serverRemoved(server)) {
serverCallbacks.remove(entry.getKey());
}
}
}
);
}
}
}

View File

@ -19,19 +19,12 @@
package com.metamx.druid.client;
import com.metamx.druid.VersionedIntervalTimeline;
import com.metamx.druid.client.selector.ServerSelector;
import com.metamx.druid.query.QueryRunner;
import java.util.concurrent.Executor;
/**
*/
public interface ServerView
{
public VersionedIntervalTimeline<String, ServerSelector> getTimeline(String dataSource);
public <T> QueryRunner<T> getQueryRunner(DruidServer server);
public void registerServerCallback(Executor exec, ServerCallback callback);
public void registerSegmentCallback(Executor exec, SegmentCallback callback);

View File

@ -19,17 +19,14 @@
package com.metamx.druid.client;
import com.metamx.druid.VersionedIntervalTimeline;
import com.metamx.druid.client.selector.ServerSelector;
import com.metamx.druid.query.QueryRunner;
/**
*/
public interface MutableServerView extends ServerView
public interface TimelineServerView extends ServerView
{
public void clear();
public void addServer(DruidServer server);
public void removeServer(DruidServer server);
public void serverAddedSegment(DruidServer server, DataSegment segment);
public void serverRemovedSegment(DruidServer server, String segmentId);
VersionedIntervalTimeline<String, ServerSelector> getTimeline(String dataSource);
<T> QueryRunner<T> getQueryRunner(DruidServer server);
}

View File

@ -1,406 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.client;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Joiner;
import com.google.common.collect.HashBasedTable;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.collect.Table;
import com.metamx.common.logger.Logger;
import com.metamx.phonebook.BasePhoneBook;
import com.metamx.phonebook.PhoneBook;
import com.metamx.phonebook.PhoneBookPeon;
import org.I0Itec.zkclient.DataUpdater;
import org.I0Itec.zkclient.IZkChildListener;
import org.I0Itec.zkclient.ZkClient;
import org.I0Itec.zkclient.exception.ZkNoNodeException;
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Executor;
/**
*/
public class ZKPhoneBook extends BasePhoneBook
{
private static final Logger log = new Logger(ZKPhoneBook.class);
public ZKPhoneBook(
final ObjectMapper jsonMapper,
final ZkClient zkClient,
final Executor peonExecutor
)
{
super(
new InternalPhoneBook(jsonMapper, zkClient, peonExecutor)
);
}
private static class InternalPhoneBook implements PhoneBook
{
private static final Joiner JOINER = Joiner.on("/");
private final Object lock = new Object();
private final Table<String, PhoneBookPeon, IZkChildListener> listeners;
private final Table<String, String, Object> announcements;
private final Map<String, PhoneBookPeon> announcementListeners;
private final ObjectMapper jsonMapper;
private final ZkClient zkClient;
private final Executor exec;
public InternalPhoneBook(
ObjectMapper jsonMapper,
ZkClient zkClient,
Executor exec
)
{
this.jsonMapper = jsonMapper;
this.zkClient = zkClient;
this.exec = exec;
listeners = HashBasedTable.create();
announcements = HashBasedTable.create();
announcementListeners = Maps.newHashMap();
}
@Override
public void start()
{
}
@Override
public void stop()
{
synchronized (lock) {
for (Map.Entry<String, PhoneBookPeon> entry : announcementListeners.entrySet()) {
unregisterListener(entry.getKey(), entry.getValue());
}
for (Table.Cell<String, String, Object> cell : announcements.cellSet()) {
unannounce(cell.getRowKey(), cell.getColumnKey());
}
}
}
@Override
public boolean isStarted()
{
return true;
}
@Override
public <T> void announce(final String serviceName, String nodeName, T properties)
{
if (!zkClient.exists(serviceName)) {
zkClient.createPersistent(serviceName, true);
}
try {
synchronized (lock) {
zkClient.createEphemeral(
getPath(serviceName, nodeName),
jsonMapper.writeValueAsString(properties)
);
PhoneBookPeon peon = announcementListeners.get(serviceName);
if (peon == null) {
peon = new PhoneBookPeon<Object>()
{
@Override
public Class<Object> getObjectClazz()
{
return Object.class;
}
@Override
public void newEntry(String name, Object properties)
{
}
@Override
public void entryRemoved(String name)
{
synchronized (lock) {
Object propertyMap = announcements.get(serviceName, name);
if (propertyMap != null) {
log.info("entry[%s/%s] was removed but I'm in charge of it, reinstating.", serviceName, name);
String path = getPath(serviceName, name);
try {
zkClient.createEphemeral(
path,
jsonMapper.writeValueAsString(propertyMap)
);
}
catch (ZkNodeExistsException e) {
log.info("Thought that [%s] didn't exist, but it did?", path);
}
catch (IOException e) {
log.error(e, "Exception thrown when recreating node[%s].", path);
}
}
}
}
};
announcementListeners.put(serviceName, peon);
registerListener(serviceName, peon);
}
announcements.put(serviceName, nodeName, properties);
}
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public void unannounce(String serviceName, String nodeName)
{
synchronized (lock) {
Object announcementMap = announcements.remove(serviceName, nodeName);
Map<String, Object> storedProperties = lookup(combineParts(Arrays.asList(serviceName, nodeName)), Map.class);
if (announcementMap == null || storedProperties == null) {
return;
}
//Hack to compute equality because jsonMapper doesn't actually give me a Map<String, String> :(
boolean areEqual = false;
try {
areEqual = storedProperties.equals(
jsonMapper.readValue(jsonMapper.writeValueAsString(announcementMap), Map.class)
);
}
catch (IOException e) {
throw new RuntimeException(e);
}
log.debug("equal?[%s]: announcementMap[%s], storedProperties[%s].", areEqual, announcementMap, storedProperties);
if (areEqual) {
zkClient.delete(getPath(serviceName, nodeName));
}
}
}
@Override
public <T> T lookup(String path, Class<? extends T> clazz)
{
final String nodeContent;
try {
nodeContent = zkClient.readData(path).toString();
}
catch (ZkNoNodeException e) {
return null;
}
try {
return jsonMapper.readValue(nodeContent, clazz);
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public <T> void post(String serviceName, String nodeName, final T properties)
{
if (!zkClient.exists(serviceName)) {
zkClient.createPersistent(serviceName);
}
final String path = getPath(serviceName, nodeName);
if (zkClient.exists(path)) {
zkClient.updateDataSerialized(
path,
new DataUpdater<Object>()
{
@Override
public Object update(Object currentData)
{
try {
return jsonMapper.writeValueAsString(properties);
}
catch (IOException e) {
log.error(e, "Exception when updating value of [%s]. Using currentData.", path);
return currentData;
}
}
}
);
}
try {
zkClient.createPersistent(path, jsonMapper.writeValueAsString(properties));
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public boolean unpost(String serviceName, String nodeName)
{
return zkClient.delete(getPath(serviceName, nodeName));
}
@Override
public <T> void postEphemeral(final String serviceName, String nodeName, T properties)
{
if (!zkClient.exists(serviceName)) {
zkClient.createPersistent(serviceName, true);
}
try {
zkClient.createEphemeral(
getPath(serviceName, nodeName),
jsonMapper.writeValueAsString(properties)
);
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public <T> void registerListener(final String serviceName, final PhoneBookPeon<T> peon)
{
final Set<String> currChildren = Sets.newHashSet();
IZkChildListener peonListener = new IZkChildListener()
{
@Override
public void handleChildChange(final String parentPath, final List<String> incomingChildren) throws Exception
{
exec.execute(
new InternalPhoneBook.UpdatingRunnable<T>(
parentPath,
currChildren,
(incomingChildren == null ? Sets.<String>newHashSet() : Sets.<String>newHashSet(incomingChildren)),
peon
)
);
}
};
zkClient.subscribeChildChanges(serviceName, peonListener);
exec.execute(
new UpdatingRunnable(serviceName, currChildren, Sets.newHashSet(zkClient.getChildren(serviceName)), peon)
);
listeners.put(serviceName, peon, peonListener);
}
@Override
public void unregisterListener(String serviceName, PhoneBookPeon peon)
{
IZkChildListener peonListener = listeners.get(serviceName, peon);
zkClient.unsubscribeChildChanges(serviceName, peonListener);
}
@Override
public String combineParts(List<String> parts)
{
return JOINER.join(parts);
}
private String getPath(String parentPath, String child)
{
return JOINER.join(parentPath, child);
}
private class UpdatingRunnable<T> implements Runnable
{
private final String serviceName;
private final Set<String> currChildren;
private final PhoneBookPeon<T> peon;
private final HashSet<String> incomingChildren;
public UpdatingRunnable(
String serviceName,
Set<String> currChildren,
final HashSet<String> incomingChildren,
PhoneBookPeon<T> peon
)
{
this.serviceName = serviceName;
this.currChildren = currChildren;
this.peon = peon;
this.incomingChildren = incomingChildren;
}
@Override
public void run()
{
try {
for (String newChild : Sets.difference(incomingChildren, currChildren)) {
log.debug(" New child[%s], for peon[%s]", newChild, peon);
String nodeContent;
try {
final String data = zkClient.readData(getPath(serviceName, newChild));
if (data != null) {
nodeContent = data.toString();
}
else {
log.error("Ignoring path[%s] with null data", getPath(serviceName, newChild));
continue;
}
}
catch (ZkNoNodeException e) {
log.info(
"Got ZkNoNodeException[%s], node must have gone bye bye before this had a chance to run.",
e.getMessage()
);
continue;
}
T nodeProperties = jsonMapper.readValue(nodeContent, peon.getObjectClazz());
peon.newEntry(newChild, nodeProperties);
currChildren.add(newChild);
}
// Sets.difference is lazy, so we have to materialize the difference before removing from the sets
Set<String> setDiff = new HashSet<String>(Sets.difference(currChildren, incomingChildren));
for (String childRemoved : setDiff) {
log.debug(" Lost child[%s], for peon[%s]", childRemoved, peon);
peon.entryRemoved(childRemoved);
currChildren.remove(childRemoved);
}
}
catch (Exception e) {
log.warn(e, "Exception thrown, serviceName[%s].", serviceName);
throw new RuntimeException(e);
}
}
}
}
}

View File

@ -26,8 +26,8 @@ import com.metamx.common.ISE;
import com.metamx.druid.client.DataSegment;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.InputStreamResponseHandler;
import com.netflix.curator.x.discovery.ServiceInstance;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.apache.curator.x.discovery.ServiceInstance;
import org.apache.curator.x.discovery.ServiceProvider;
import org.joda.time.Interval;
import java.io.InputStream;

View File

@ -0,0 +1,121 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.coordination;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.curator.announcement.Announcer;
import com.metamx.druid.initialization.ZkPathsConfig;
import org.apache.curator.utils.ZKPaths;
import java.io.IOException;
public class CuratorDataSegmentAnnouncer implements DataSegmentAnnouncer
{
private static final Logger log = new Logger(CuratorDataSegmentAnnouncer.class);
private final Object lock = new Object();
private final DruidServerMetadata server;
private final ZkPathsConfig config;
private final Announcer announcer;
private final ObjectMapper jsonMapper;
private final String servedSegmentsLocation;
private volatile boolean started = false;
public CuratorDataSegmentAnnouncer(
DruidServerMetadata server,
ZkPathsConfig config,
Announcer announcer,
ObjectMapper jsonMapper
)
{
this.server = server;
this.config = config;
this.announcer = announcer;
this.jsonMapper = jsonMapper;
this.servedSegmentsLocation = ZKPaths.makePath(config.getServedSegmentsPath(), server.getName());
}
@LifecycleStart
public void start()
{
synchronized (lock) {
if (started) {
return;
}
try {
final String path = makeAnnouncementPath();
log.info("Announcing self[%s] at [%s]", server, path);
announcer.announce(path, jsonMapper.writeValueAsBytes(server));
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
}
started = true;
}
}
@LifecycleStop
public void stop()
{
synchronized (lock) {
if (!started) {
return;
}
log.info("Stopping CuratorDataSegmentAnnouncer with config[%s]", config);
announcer.unannounce(makeAnnouncementPath());
started = false;
}
}
public void announceSegment(DataSegment segment) throws IOException
{
final String path = makeServedSegmentPath(segment);
log.info("Announcing segment[%s] to path[%s]", segment.getIdentifier(), path);
announcer.announce(path, jsonMapper.writeValueAsBytes(segment));
}
public void unannounceSegment(DataSegment segment) throws IOException
{
final String path = makeServedSegmentPath(segment);
log.info("Unannouncing segment[%s] at path[%s]", segment.getIdentifier(), path);
announcer.unannounce(path);
}
private String makeAnnouncementPath() {
return ZKPaths.makePath(config.getAnnouncementsPath(), server.getName());
}
private String makeServedSegmentPath(DataSegment segment)
{
return ZKPaths.makePath(servedSegmentsLocation, segment.getIdentifier());
}
}

View File

@ -1,10 +1,10 @@
package com.metamx.druid.realtime;
package com.metamx.druid.coordination;
import com.metamx.druid.client.DataSegment;
import java.io.IOException;
public interface SegmentAnnouncer
public interface DataSegmentAnnouncer
{
public void announceSegment(DataSegment segment) throws IOException;
public void unannounceSegment(DataSegment segment) throws IOException;

View File

@ -0,0 +1,91 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.coordination;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
/**
*/
public class DruidServerMetadata
{
private final String name;
private final String host;
private final long maxSize;
private final String tier;
private final String type;
@JsonCreator
public DruidServerMetadata(
@JsonProperty("name") String name,
@JsonProperty("host") String host,
@JsonProperty("maxSize") long maxSize,
@JsonProperty("type") String type, @JsonProperty("tier") String tier
)
{
this.name = name;
this.host = host;
this.maxSize = maxSize;
this.tier = tier;
this.type = type;
}
@JsonProperty
public String getName()
{
return name;
}
@JsonProperty
public String getHost()
{
return host;
}
@JsonProperty
public long getMaxSize()
{
return maxSize;
}
@JsonProperty
public String getTier()
{
return tier;
}
@JsonProperty
public String getType()
{
return type;
}
@Override
public String toString()
{
return "DruidServer{" +
"name='" + name + '\'' +
", host='" + host + '\'' +
", maxSize=" + maxSize +
", tier='" + tier + '\'' +
", type='" + type + '\'' +
'}';
}
}

View File

@ -0,0 +1,56 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.curator;
import org.apache.curator.framework.api.CompressionProvider;
import org.apache.curator.framework.imps.GzipCompressionProvider;
import java.io.IOException;
import java.util.zip.ZipException;
/**
*/
public class PotentiallyGzippedCompressionProvider implements CompressionProvider
{
private final boolean compressOutput;
GzipCompressionProvider base = new GzipCompressionProvider();
public PotentiallyGzippedCompressionProvider(boolean compressOutput)
{
this.compressOutput = compressOutput;
}
@Override
public byte[] compress(String path, byte[] data) throws Exception
{
return compressOutput ? base.compress(path, data) : data;
}
@Override
public byte[] decompress(String path, byte[] data) throws Exception
{
try {
return base.decompress(path, data);
}
catch (IOException e) {
return data;
}
}
}

View File

@ -0,0 +1,115 @@
package com.metamx.druid.curator;
import com.google.common.collect.ImmutableList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
/**
* This class exists to ignore the shutdownNow() call that PathChildrenCache does on close() so that we can share the
* same executor amongst multiple caches...
*/
public class ShutdownNowIgnoringExecutorService implements ExecutorService
{
private final ExecutorService exec;
public ShutdownNowIgnoringExecutorService(
ExecutorService exec
)
{
this.exec = exec;
}
@Override
public void shutdown()
{
// Ignore!
}
@Override
public List<Runnable> shutdownNow()
{
// Ignore!
return ImmutableList.of();
}
@Override
public boolean isShutdown()
{
return exec.isShutdown();
}
@Override
public boolean isTerminated()
{
return exec.isTerminated();
}
@Override
public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException
{
return exec.awaitTermination(timeout, unit);
}
@Override
public <T> Future<T> submit(Callable<T> task)
{
return exec.submit(task);
}
@Override
public <T> Future<T> submit(Runnable task, T result)
{
return exec.submit(task, result);
}
@Override
public Future<?> submit(Runnable task)
{
return exec.submit(task);
}
@Override
public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks) throws InterruptedException
{
return exec.invokeAll(tasks);
}
@Override
public <T> List<Future<T>> invokeAll(
Collection<? extends Callable<T>> tasks,
long timeout,
TimeUnit unit
) throws InterruptedException
{
return exec.invokeAll(tasks, timeout, unit);
}
@Override
public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
{
return exec.invokeAny(tasks);
}
@Override
public <T> T invokeAny(
Collection<? extends Callable<T>> tasks,
long timeout,
TimeUnit unit
) throws InterruptedException, ExecutionException, TimeoutException
{
return exec.invokeAny(tasks, timeout, unit);
}
@Override
public void execute(Runnable command)
{
exec.execute(command);
}
}

View File

@ -0,0 +1,268 @@
package com.metamx.druid.curator.announcement;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.collect.MapMaker;
import com.google.common.collect.Sets;
import com.google.common.io.Closeables;
import com.metamx.common.IAE;
import com.metamx.common.Pair;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.curator.ShutdownNowIgnoringExecutorService;
import com.metamx.druid.curator.cache.PathChildrenCacheFactory;
import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.ChildData;
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.curator.utils.ZKPaths;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicReference;
/**
* Announces things on Zookeeper.
*/
public class Announcer
{
private static final Logger log = new Logger(Announcer.class);
private final CuratorFramework curator;
private final PathChildrenCacheFactory factory;
private final List<Pair<String, byte[]>> toAnnounce = Lists.newArrayList();
private final ConcurrentMap<String, PathChildrenCache> listeners = new MapMaker().makeMap();
private final ConcurrentMap<String, ConcurrentMap<String, byte[]>> announcements = new MapMaker().makeMap();
private boolean started = false;
public Announcer(
CuratorFramework curator,
ExecutorService exec
)
{
this.curator = curator;
this.factory = new SimplePathChildrenCacheFactory(false, true, new ShutdownNowIgnoringExecutorService(exec));
}
@LifecycleStart
public void start()
{
synchronized (toAnnounce) {
if (started) {
return;
}
started = true;
for (Pair<String, byte[]> pair : toAnnounce) {
announce(pair.lhs, pair.rhs);
}
toAnnounce.clear();
}
}
@LifecycleStop
public void stop()
{
synchronized (toAnnounce) {
if (!started) {
return;
}
started = false;
for (Map.Entry<String, PathChildrenCache> entry : listeners.entrySet()) {
Closeables.closeQuietly(entry.getValue());
}
for (Map.Entry<String, ConcurrentMap<String, byte[]>> entry : announcements.entrySet()) {
String basePath = entry.getKey();
for (String announcementPath : entry.getValue().keySet()) {
unannounce(ZKPaths.makePath(basePath, announcementPath));
}
}
}
}
/**
* Announces the provided bytes at the given path. Announcement means that it will create an ephemeral node
* and monitor it to make sure that it always exists until it is unannounced or this object is closed.
*
* @param path The path to announce at
* @param bytes The payload to announce
*/
public void announce(String path, byte[] bytes)
{
synchronized (toAnnounce) {
if (!started) {
toAnnounce.add(Pair.of(path, bytes));
return;
}
}
final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
final String parentPath = pathAndNode.getPath();
ConcurrentMap<String, byte[]> subPaths = announcements.get(parentPath);
if (subPaths == null) {
// I don't have a watcher on this path yet, create a Map and start watching.
announcements.putIfAbsent(parentPath, new MapMaker().<String, byte[]>makeMap());
// Guaranteed to be non-null, but might be a map put in there by another thread.
final ConcurrentMap<String, byte[]> finalSubPaths = announcements.get(parentPath);
// Synchronize to make sure that I only create a listener once.
synchronized (finalSubPaths) {
if (! listeners.containsKey(parentPath)) {
final PathChildrenCache cache = factory.make(curator, parentPath);
cache.getListenable().addListener(
new PathChildrenCacheListener()
{
private final AtomicReference<Set<String>> pathsLost = new AtomicReference<Set<String>>(null);
@Override
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
{
log.info("Path[%s] got event[%s]", parentPath, event);
switch (event.getType()) {
case CHILD_REMOVED:
final ChildData child = event.getData();
final ZKPaths.PathAndNode childPath = ZKPaths.getPathAndNode(child.getPath());
final byte[] value = finalSubPaths.get(childPath.getNode());
if (value != null) {
log.info("Node[%s] dropped, reinstating.", child.getPath());
createAnnouncement(child.getPath(), value);
}
break;
case CONNECTION_LOST:
// Lost connection, which means session is broken, take inventory of what has been seen.
// This is to protect from a race condition in which the ephemeral node could have been
// created but not actually seen by the PathChildrenCache, which means that it won't know
// that it disappeared and thus will not generate a CHILD_REMOVED event for us. Under normal
// circumstances, this can only happen upon connection loss; but technically if you have
// an adversary in the system, they could also delete the ephemeral node before the cache sees
// it. This does not protect from that case, so don't have adversaries.
Set<String> pathsToReinstate = Sets.newHashSet();
for (String node : finalSubPaths.keySet()) {
pathsToReinstate.add(ZKPaths.makePath(parentPath, node));
}
for (ChildData data : cache.getCurrentData()) {
pathsToReinstate.remove(data.getPath());
}
if (!pathsToReinstate.isEmpty() && !pathsLost.compareAndSet(null, pathsToReinstate)) {
log.info("Already had a pathsLost set!?[%s]", parentPath);
}
break;
case CONNECTION_RECONNECTED:
final Set<String> thePathsLost = pathsLost.getAndSet(null);
if (thePathsLost != null) {
for (String path : thePathsLost) {
log.info("Reinstating [%s]", path);
final ZKPaths.PathAndNode split = ZKPaths.getPathAndNode(path);
createAnnouncement(path, announcements.get(split.getPath()).get(split.getNode()));
}
}
break;
}
}
}
);
try {
synchronized (toAnnounce) {
if (started) {
cache.start();
listeners.put(parentPath, cache);
}
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
subPaths = finalSubPaths;
}
boolean created = false;
synchronized (toAnnounce) {
if (started) {
byte[] oldBytes = subPaths.putIfAbsent(pathAndNode.getNode(), bytes);
if (oldBytes != null) {
throw new IAE("Already announcing[%s], cannot announce it twice.", path);
}
created = true;
}
}
if (created) {
try {
createAnnouncement(path, bytes);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
private String createAnnouncement(final String path, byte[] value) throws Exception
{
return curator.create().compressed().withMode(CreateMode.EPHEMERAL).inBackground().forPath(path, value);
}
/**
* Unannounces an announcement created at path. Note that if all announcements get removed, the Announcer
* will continue to have ZK watches on paths because clearing them out is a source of ugly race conditions.
*
* If you need to completely clear all the state of what is being watched and announced, stop() the Announcer.
*
* @param path
*/
public void unannounce(String path)
{
log.info("unannouncing [%s]", path);
final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
final String parentPath = pathAndNode.getPath();
final ConcurrentMap<String, byte[]> subPaths = announcements.get(parentPath);
if (subPaths == null) {
throw new IAE("Path[%s] not announced, cannot unannounce.", path);
}
if (subPaths.remove(pathAndNode.getNode()) == null) {
throw new IAE("Path[%s] not announced, cannot unannounce.", path);
}
try {
curator.delete().guaranteed().forPath(path);
}
catch (KeeperException.NoNodeException e) {
log.info("node[%s] didn't exist anyway...", path);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -17,20 +17,14 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.initialization;
package com.metamx.druid.curator.cache;
import org.skife.config.Config;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
/**
*/
public abstract class ZkClientConfig
public interface PathChildrenCacheFactory
{
@Config("druid.zk.service.host")
public abstract String getZkHosts();
@Config("druid.zk.service.connectionTimeout")
public int getConnectionTimeout()
{
return Integer.MAX_VALUE;
}
public PathChildrenCache make(CuratorFramework curator, String path);
}

View File

@ -17,32 +17,35 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.zk;
package com.metamx.druid.curator.cache;
import com.metamx.common.IAE;
import org.I0Itec.zkclient.exception.ZkMarshallingError;
import org.I0Itec.zkclient.serialize.ZkSerializer;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import java.nio.charset.Charset;
import java.util.concurrent.ExecutorService;
/**
*/
public class StringZkSerializer implements ZkSerializer
*/
public class SimplePathChildrenCacheFactory implements PathChildrenCacheFactory
{
private static final Charset UTF8 = Charset.forName("UTF8");
private final boolean cacheData;
private final boolean compressed;
private final ExecutorService exec;
@Override
public byte[] serialize(Object data) throws ZkMarshallingError
public SimplePathChildrenCacheFactory(
boolean cacheData,
boolean compressed,
ExecutorService exec
)
{
if (data instanceof String) {
return ((String) data).getBytes(UTF8);
}
throw new IAE("Can only serialize strings into ZK");
this.cacheData = cacheData;
this.compressed = compressed;
this.exec = exec;
}
@Override
public Object deserialize(byte[] bytes) throws ZkMarshallingError
public PathChildrenCache make(CuratorFramework curator, String path)
{
return new String(bytes, UTF8);
return new PathChildrenCache(curator, path, cacheData, compressed, exec);
}
}

View File

@ -0,0 +1,302 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.curator.inventory;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.MapMaker;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.curator.ShutdownNowIgnoringExecutorService;
import com.metamx.druid.curator.cache.PathChildrenCacheFactory;
import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.ChildData;
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.curator.utils.ZKPaths;
import java.io.IOException;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicReference;
/**
* An InventoryManager watches updates to inventory on Zookeeper (or some other discovery-like service publishing
* system). It is built up on two object types: containers and inventory objects.
*
* The logic of the InventoryManager just maintains a local cache of the containers and inventory it sees on ZK. It
* provides methods for getting at the container objects, which house the actual individual pieces of inventory.
*
* A Strategy is provided to the constructor of an Inventory manager, this strategy provides all of the
* object-specific logic to serialize, deserialize, compose and alter the container and inventory objects.
*/
public class CuratorInventoryManager<ContainerClass, InventoryClass>
{
private static final Logger log = new Logger(CuratorInventoryManager.class);
private final Object lock = new Object();
private final CuratorFramework curatorFramework;
private final InventoryManagerConfig config;
private final CuratorInventoryManagerStrategy<ContainerClass, InventoryClass> strategy;
private final ConcurrentMap<String, ContainerHolder> containers;
private final PathChildrenCacheFactory cacheFactory;
private volatile PathChildrenCache childrenCache;
public CuratorInventoryManager(
CuratorFramework curatorFramework,
InventoryManagerConfig config,
ExecutorService exec,
CuratorInventoryManagerStrategy<ContainerClass, InventoryClass> strategy
)
{
this.curatorFramework = curatorFramework;
this.config = config;
this.strategy = strategy;
this.containers = new MapMaker().makeMap();
this.cacheFactory = new SimplePathChildrenCacheFactory(true, true, new ShutdownNowIgnoringExecutorService(exec));
}
@LifecycleStart
public void start() throws Exception
{
synchronized (lock) {
if (childrenCache != null) {
return;
}
childrenCache = cacheFactory.make(curatorFramework, config.getContainerPath());
}
childrenCache.getListenable().addListener(new ContainerCacheListener());
try {
childrenCache.start();
}
catch (Exception e) {
synchronized (lock) {
try {
stop();
}
catch (IOException e1) {
log.error(e1, "Exception when stopping InventoryManager that couldn't start.");
}
}
throw e;
}
}
@LifecycleStop
public void stop() throws IOException
{
synchronized (lock) {
if (childrenCache == null) {
return;
}
// This close() call actually calls shutdownNow() on the executor registered with the Cache object...
childrenCache.close();
childrenCache = null;
}
for (String containerKey : Lists.newArrayList(containers.keySet())) {
final ContainerHolder containerHolder = containers.remove(containerKey);
if (containerHolder == null) {
log.wtf("!? Got key[%s] from keySet() but it didn't have a value!?", containerKey);
}
else {
// This close() call actually calls shutdownNow() on the executor registered with the Cache object...
containerHolder.getCache().close();
}
}
}
public ContainerClass getInventoryValue(String containerKey)
{
final ContainerHolder containerHolder = containers.get(containerKey);
return containerHolder == null ? null : containerHolder.getContainer();
}
public Iterable<ContainerClass> getInventory()
{
return Iterables.transform(
containers.values(),
new Function<ContainerHolder, ContainerClass>()
{
@Override
public ContainerClass apply(ContainerHolder input)
{
return input.getContainer();
}
}
);
}
private class ContainerHolder
{
private final AtomicReference<ContainerClass> container;
private final PathChildrenCache cache;
ContainerHolder(
ContainerClass container,
PathChildrenCache cache
)
{
this.container = new AtomicReference<ContainerClass>(container);
this.cache = cache;
}
private ContainerClass getContainer()
{
return container.get();
}
private void setContainer(ContainerClass newContainer)
{
container.set(newContainer);
}
private PathChildrenCache getCache()
{
return cache;
}
}
private class ContainerCacheListener implements PathChildrenCacheListener
{
@Override
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
{
final ChildData child = event.getData();
if (child == null) {
return;
}
final String containerKey = ZKPaths.getNodeFromPath(child.getPath());
final ContainerClass container;
switch (event.getType()) {
case CHILD_ADDED:
container = strategy.deserializeContainer(child.getData());
// This would normally be a race condition, but the only thing that should be mutating the containers
// map is this listener, which should never run concurrently. If the same container is going to disappear
// and come back, we expect a removed event in between.
if (containers.containsKey(containerKey)) {
log.error("New node[%s] but there was already one. That's not good, ignoring new one.", child.getPath());
}
final String inventoryPath = String.format("%s/%s", config.getInventoryPath(), containerKey);
PathChildrenCache inventoryCache = cacheFactory.make(curatorFramework, inventoryPath);
inventoryCache.getListenable().addListener(new InventoryCacheListener(containerKey, inventoryPath));
containers.put(containerKey, new ContainerHolder(container, inventoryCache));
inventoryCache.start();
strategy.newContainer(container);
break;
case CHILD_REMOVED:
final ContainerHolder removed = containers.remove(containerKey);
if (removed == null) {
log.warn("Container[%s] removed that wasn't a container!?", child.getPath());
break;
}
// This close() call actually calls shutdownNow() on the executor registered with the Cache object, it
// better have its own executor or ignore shutdownNow() calls...
removed.getCache().close();
strategy.deadContainer(removed.getContainer());
break;
case CHILD_UPDATED:
container = strategy.deserializeContainer(child.getData());
ContainerHolder oldContainer = containers.get(containerKey);
if (oldContainer == null) {
log.warn("Container update[%s], but the old container didn't exist!? Ignoring.", child.getPath());
}
else {
synchronized (oldContainer) {
oldContainer.setContainer(strategy.updateContainer(oldContainer.getContainer(), container));
}
}
break;
}
}
private class InventoryCacheListener implements PathChildrenCacheListener
{
private final String containerKey;
private final String inventoryPath;
public InventoryCacheListener(String containerKey, String inventoryPath)
{
this.containerKey = containerKey;
this.inventoryPath = inventoryPath;
}
@Override
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
{
final ChildData child = event.getData();
if (child == null) {
return;
}
final ContainerHolder holder = containers.get(containerKey);
if (holder == null) {
return;
}
final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath());
switch (event.getType()) {
case CHILD_ADDED:
case CHILD_UPDATED:
final InventoryClass inventory = strategy.deserializeInventory(child.getData());
synchronized (holder) {
holder.setContainer(strategy.addInventory(holder.getContainer(), inventoryKey, inventory));
}
break;
case CHILD_REMOVED:
synchronized (holder) {
holder.setContainer(strategy.removeInventory(holder.getContainer(), inventoryKey));
}
break;
}
}
}
}
}

View File

@ -17,22 +17,21 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.client;
import com.metamx.common.Pair;
import com.metamx.phonebook.PhoneBookPeon;
import java.util.Map;
package com.metamx.druid.curator.inventory;
/**
*/
public interface InventoryManagementStrategy<T>
public interface CuratorInventoryManagerStrategy<ContainerClass, InventoryClass>
{
public Class<T> getContainerClass();
public Pair<String, PhoneBookPeon<?>> makeSubListener(final T baseObject);
public void objectRemoved(final T baseObject);
public ContainerClass deserializeContainer(byte[] bytes);
public byte[] serializeContainer(ContainerClass container);
// These are a hack to get around a poor serialization choice, please do not use
public boolean doesSerde();
public T deserialize(String name, Map<String, String> properties);
public InventoryClass deserializeInventory(byte[] bytes);
public byte[] serializeInventory(InventoryClass inventory);
public void newContainer(ContainerClass newContainer);
public void deadContainer(ContainerClass deadContainer);
public ContainerClass updateContainer(ContainerClass oldContainer, ContainerClass newContainer);
public ContainerClass addInventory(ContainerClass container, String inventoryKey, InventoryClass inventory);
public ContainerClass removeInventory(ContainerClass container, String inventoryKey);
}

View File

@ -17,54 +17,37 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.client;
package com.metamx.druid.curator.inventory;
/**
*/
public class InventoryManagerConfig
public interface InventoryManagerConfig
{
private final String inventoryIdPath;
private final String inventoryPath;
public InventoryManagerConfig(
String inventoryIdPath,
String inventoryPath
)
{
this.inventoryIdPath = inventoryIdPath;
this.inventoryPath = inventoryPath;
}
/**
* The InventoryIdPath is the path where the InventoryManager should look for new inventory buckets. Because ZK does
* not allow for children under ephemeral nodes, the common interaction for registering Inventory that might be
* ephemeral is to
* The ContainerPath is the path where the InventoryManager should look for new containers of inventory.
*
* Because ZK does not allow for children under ephemeral nodes, the common interaction for registering Inventory
* that might be ephemeral is to
*
* 1) Create a permanent node underneath the InventoryPath
* 2) Create an ephemeral node underneath the InventoryIdPath with the same name as the permanent nodes under InventoryPath
* 2) Create an ephemeral node under the ContainerPath with the same name as the permanent node under InventoryPath
* 3) For each piece of "inventory", create an ephemeral node as a child of the node created in step (1)
*
* @return the inventoryIdPath
* @return the containerPath
*/
public String getInventoryIdPath()
{
return inventoryIdPath;
}
public String getContainerPath();
/**
* The InventoryPath is the path where the InventoryManager should look for new inventory.
*
* Because ZK does not allow for children under ephemeral nodes, the common interaction for registering an Inventory
* container that might be ephemeral (like a server) is to
* Because ZK does not allow for children under ephemeral nodes, the common interaction for registering Inventory
* that might be ephemeral is to
*
* 1) Create a permanent node underneath the InventoryPath
* 2) Create an ephemeral node underneath the InventoryIdPath with the same name as the permanent nodes under InventoryPath
* 2) Create an ephemeral node under the ContainerPath with the same name as the permanent node under InventoryPath
* 3) For each piece of "inventory", create an ephemeral node as a child of the node created in step (1)
*
* @return the inventoryIdPath
* @return the inventoryPath
*/
public String getInventoryPath()
{
return inventoryPath;
}
public String getInventoryPath();
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.Module;
@ -34,8 +35,6 @@ import com.metamx.common.logger.Logger;
import com.metamx.druid.QueryableNode;
import com.metamx.druid.client.BrokerServerView;
import com.metamx.druid.client.CachingClusteredClient;
import com.metamx.druid.client.ClientConfig;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.cache.Cache;
import com.metamx.druid.client.cache.CacheConfig;
import com.metamx.druid.client.cache.CacheMonitor;
@ -53,15 +52,16 @@ import com.metamx.http.client.HttpClient;
import com.metamx.http.client.HttpClientConfig;
import com.metamx.http.client.HttpClientInit;
import com.metamx.metrics.Monitor;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.x.discovery.ServiceDiscovery;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.x.discovery.ServiceDiscovery;
import org.mortbay.jetty.servlet.Context;
import org.mortbay.jetty.servlet.ServletHolder;
import org.skife.config.ConfigurationObjectFactory;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
/**
*/
@ -96,7 +96,7 @@ public class BrokerNode extends QueryableNode<BrokerNode>
ConfigurationObjectFactory configFactory
)
{
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
super("broker", log, props, lifecycle, jsonMapper, smileMapper, configFactory);
}
public QueryToolChestWarehouse getWarehouse()
@ -189,11 +189,12 @@ public class BrokerNode extends QueryableNode<BrokerNode>
monitors.add(new CacheMonitor(cache));
startMonitoring(monitors);
final BrokerServerView view = new BrokerServerView(warehouse, getSmileMapper(), brokerHttpClient);
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
getConfigFactory().build(ClientConfig.class), getPhoneBook(), view
final ExecutorService viewExec = Executors.newFixedThreadPool(
1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("BrokerServerView-%s").build()
);
final BrokerServerView view = new BrokerServerView(
warehouse, getSmileMapper(), brokerHttpClient, getServerInventoryView(), viewExec
);
lifecycle.addManagedInstance(clientInventoryManager);
final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cache, getSmileMapper());
lifecycle.addManagedInstance(baseClient);
@ -201,7 +202,7 @@ public class BrokerNode extends QueryableNode<BrokerNode>
final ClientQuerySegmentWalker texasRanger = new ClientQuerySegmentWalker(warehouse, getEmitter(), baseClient);
List<Module> theModules = Lists.newArrayList();
theModules.add(new ClientServletModule(texasRanger, clientInventoryManager, getJsonMapper()));
theModules.add(new ClientServletModule(texasRanger, getServerInventoryView(), getJsonMapper()));
theModules.addAll(extraModules);
final Injector injector = Guice.createInjector(theModules);
@ -226,7 +227,7 @@ public class BrokerNode extends QueryableNode<BrokerNode>
final Lifecycle lifecycle = getLifecycle();
final ServiceDiscoveryConfig serviceDiscoveryConfig = getConfigFactory().build(ServiceDiscoveryConfig.class);
CuratorFramework curatorFramework = Initialization.makeCuratorFrameworkClient(
CuratorFramework curatorFramework = Initialization.makeCuratorFramework(
serviceDiscoveryConfig, lifecycle
);

View File

@ -24,10 +24,10 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.DruidDataSource;
import com.metamx.druid.client.DruidServer;
import com.metamx.druid.client.ServerInventoryView;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -47,20 +47,20 @@ public class ClientInfoResource
{
private static final int SEGMENT_HISTORY_MILLIS = 7 * 24 * 60 * 60 * 1000; // ONE WEEK
private ClientInventoryManager clientInventoryManager;
private ServerInventoryView serverInventoryView;
@Inject
public ClientInfoResource(
ClientInventoryManager clientInventoryManager
ServerInventoryView serverInventoryView
)
{
this.clientInventoryManager = clientInventoryManager;
this.serverInventoryView = serverInventoryView;
}
private Map<String, List<DataSegment>> getSegmentsForDatasources()
{
final Map<String, List<DataSegment>> dataSourceMap = Maps.newHashMap();
for (DruidServer server : clientInventoryManager.getInventory()) {
for (DruidServer server : serverInventoryView.getInventory()) {
for (DruidDataSource dataSource : server.getDataSources()) {
if (!dataSourceMap.containsKey(dataSource.getName())) {
dataSourceMap.put(dataSource.getName(), Lists.<DataSegment>newArrayList());

View File

@ -22,7 +22,7 @@ package com.metamx.druid.http;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
import com.google.inject.Provides;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.ServerInventoryView;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.sun.jersey.guice.JerseyServletModule;
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
@ -34,17 +34,17 @@ import javax.inject.Singleton;
public class ClientServletModule extends JerseyServletModule
{
private final QuerySegmentWalker texasRanger;
private final ClientInventoryManager clientInventoryManager;
private final ServerInventoryView serverInventoryView;
private final ObjectMapper jsonMapper;
public ClientServletModule(
QuerySegmentWalker texasRanger,
ClientInventoryManager clientInventoryManager,
ServerInventoryView serverInventoryView,
ObjectMapper jsonMapper
)
{
this.texasRanger = texasRanger;
this.clientInventoryManager = clientInventoryManager;
this.serverInventoryView = serverInventoryView;
this.jsonMapper = jsonMapper;
}
@ -53,7 +53,7 @@ public class ClientServletModule extends JerseyServletModule
{
bind(ClientInfoResource.class);
bind(QuerySegmentWalker.class).toInstance(texasRanger);
bind(ClientInventoryManager.class).toInstance(clientInventoryManager);
bind(ServerInventoryView.class).toInstance(serverInventoryView);
serve("/*").with(GuiceContainer.class);
}

View File

@ -30,6 +30,6 @@ public abstract class CuratorConfig
public abstract String getZkHosts();
@Config("druid.zk.service.sessionTimeoutMs")
@Default("15000")
@Default("30000")
public abstract int getZkSessionTimeoutMs();
}

View File

@ -20,39 +20,39 @@
package com.metamx.druid.initialization;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.google.common.io.Closeables;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.config.Config;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.ZKPhoneBook;
import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider;
import com.metamx.druid.http.EmittingRequestLogger;
import com.metamx.druid.http.FileRequestLogger;
import com.metamx.druid.http.RequestLogger;
import com.metamx.druid.utils.PropUtils;
import com.metamx.druid.zk.PropertiesZkSerializer;
import com.metamx.druid.zk.StringZkSerializer;
import com.metamx.emitter.core.Emitter;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.CuratorFrameworkFactory;
import com.netflix.curator.retry.ExponentialBackoffRetry;
import com.netflix.curator.x.discovery.ServiceDiscovery;
import com.netflix.curator.x.discovery.ServiceDiscoveryBuilder;
import com.netflix.curator.x.discovery.ServiceInstance;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.I0Itec.zkclient.ZkClient;
import org.I0Itec.zkclient.ZkConnection;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.BoundedExponentialBackoffRetry;
import org.apache.curator.x.discovery.ServiceDiscovery;
import org.apache.curator.x.discovery.ServiceDiscoveryBuilder;
import org.apache.curator.x.discovery.ServiceInstance;
import org.apache.curator.x.discovery.ServiceProvider;
import org.apache.zookeeper.data.Stat;
import org.mortbay.jetty.Connector;
import org.mortbay.jetty.Server;
import org.mortbay.jetty.nio.SelectChannelConnector;
import org.mortbay.thread.QueuedThreadPool;
import org.skife.config.ConfigurationObjectFactory;
import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.util.Properties;
import java.util.concurrent.Executors;
/**
*/
@ -60,70 +60,23 @@ public class Initialization
{
private static final Logger log = new Logger(Initialization.class);
private static final String PROPERTIES_FILE = "runtime.properties";
private static final Properties zkProps = new Properties();
private static final Properties fileProps = new Properties(zkProps);
private static Properties props = null;
public final static String PROP_SUBPATH = "properties";
public final static String[] SUB_PATHS = {"announcements", "servedSegments", "loadQueue", "master"};
public final static String[] SUB_PATH_PROPS = {
"druid.zk.paths.announcementsPath",
"druid.zk.paths.servedSegmentsPath",
"druid.zk.paths.loadQueuePath",
"druid.zk.paths.masterPath"
};
public static final String DEFAULT_ZPATH = "/druid";
public static ZkClient makeZkClient(ZkClientConfig config, Lifecycle lifecycle)
{
final ZkClient retVal = new ZkClient(
new ZkConnection(config.getZkHosts()),
config.getConnectionTimeout(),
new StringZkSerializer()
);
lifecycle.addHandler(
new Lifecycle.Handler()
{
@Override
public void start() throws Exception
{
retVal.waitUntilConnected();
}
@Override
public void stop()
{
retVal.close();
}
}
);
return retVal;
}
public static ZKPhoneBook createPhoneBook(
ObjectMapper jsonMapper, ZkClient zkClient, String threadNameFormat, Lifecycle lifecycle
)
{
return lifecycle.addManagedInstance(
new ZKPhoneBook(
jsonMapper,
zkClient,
Executors.newSingleThreadExecutor(
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat(threadNameFormat)
.build()
)
)
);
}
/**
* Load properties.
* Properties are layered, high to low precedence: cmdLine -D, runtime.properties file, stored in zookeeper.
* Properties are layered:
*
* # stored in zookeeper
* # runtime.properties file,
* # cmdLine -D
*
* command line overrides runtime.properties which overrides zookeeper
*
* Idempotent. Thread-safe. Properties are only loaded once.
* If property druid.zk.service.host=none then do not load properties from zookeeper.
* If property druid.zk.service.host is not set then do not load properties from zookeeper.
*
* @return Properties ready to use.
*/
@ -139,13 +92,11 @@ public class Initialization
Properties tmp_props = new Properties(fileProps); // the head of the 3 level Properties chain
tmp_props.putAll(sp);
final InputStream stream = ClassLoader.getSystemResourceAsStream("runtime.properties");
final InputStream stream = ClassLoader.getSystemResourceAsStream(PROPERTIES_FILE);
if (stream == null) {
log.info(
"runtime.properties not found as a resource in classpath, relying only on system properties, and zookeeper now."
);
log.info("%s not found on classpath, relying only on system properties and zookeeper.", PROPERTIES_FILE);
} else {
log.info("Loading properties from runtime.properties");
log.info("Loading properties from %s", PROPERTIES_FILE);
try {
try {
fileProps.load(stream);
@ -159,58 +110,46 @@ public class Initialization
}
}
// log properties from file; note stringPropertyNames() will follow Properties.defaults but
// next level is empty at this point.
// log properties from file; stringPropertyNames() would normally cascade down into the sub Properties objects, but
// zkProps (the parent level) is empty at this point so it will only log properties from runtime.properties
for (String prop : fileProps.stringPropertyNames()) {
log.info("Loaded(runtime.properties) Property[%s] as [%s]", prop, fileProps.getProperty(prop));
}
final String zk_hosts = tmp_props.getProperty("druid.zk.service.host");
final String zkHostsProperty = "druid.zk.service.host";
if (zk_hosts != null) {
if (!zk_hosts.equals("none")) { // get props from zk
final ZkClient zkPropLoadingClient;
final ZkClientConfig clientConfig = new ZkClientConfig()
{
@Override
public String getZkHosts()
{
return zk_hosts;
}
};
if (tmp_props.getProperty(zkHostsProperty) != null) {
final ConfigurationObjectFactory factory = Config.createFactory(tmp_props);
zkPropLoadingClient = new ZkClient(
new ZkConnection(clientConfig.getZkHosts()),
clientConfig.getConnectionTimeout(),
new PropertiesZkSerializer()
);
zkPropLoadingClient.waitUntilConnected();
String propertiesZNodePath = tmp_props.getProperty("druid.zk.paths.propertiesPath");
if (propertiesZNodePath == null) {
String zpathBase = tmp_props.getProperty("druid.zk.paths.base", DEFAULT_ZPATH);
propertiesZNodePath = makePropPath(zpathBase);
}
// get properties stored by zookeeper (lowest precedence)
if (zkPropLoadingClient.exists(propertiesZNodePath)) {
Properties p = zkPropLoadingClient.readData(propertiesZNodePath, true);
if (p != null) {
zkProps.putAll(p);
}
Lifecycle lifecycle = new Lifecycle();
try {
final ZkPathsConfig config = factory.build(ZkPathsConfig.class);
CuratorFramework curator = makeCuratorFramework(factory.build(CuratorConfig.class), lifecycle);
lifecycle.start();
final Stat stat = curator.checkExists().forPath(config.getPropertiesPath());
if (stat != null) {
final byte[] data = curator.getData().forPath(config.getPropertiesPath());
zkProps.load(new InputStreamReader(new ByteArrayInputStream(data), Charsets.UTF_8));
}
// log properties from zk
for (String prop : zkProps.stringPropertyNames()) {
log.info("Loaded(properties stored in zk) Property[%s] as [%s]", prop, zkProps.getProperty(prop));
log.info("Loaded(zk) Property[%s] as [%s]", prop, zkProps.getProperty(prop));
}
} // get props from zk
}
catch (Exception e) {
throw Throwables.propagate(e);
}
finally {
lifecycle.stop();
}
} else {
log.warn("property druid.zk.service.host is not set, so no way to contact zookeeper for coordination.");
log.warn("property[%s] not set, skipping ZK-specified properties.", zkHostsProperty);
}
// validate properties now that all levels of precedence are loaded
if (!validateResolveProps(tmp_props)) {
log.error("Properties failed to validate, cannot continue");
throw new RuntimeException("Properties failed to validate");
}
props = tmp_props; // publish
props = tmp_props;
return props;
}
@ -234,7 +173,7 @@ public class Initialization
return server;
}
public static CuratorFramework makeCuratorFrameworkClient(
public static CuratorFramework makeCuratorFramework(
CuratorConfig curatorConfig,
Lifecycle lifecycle
) throws IOException
@ -242,12 +181,10 @@ public class Initialization
final CuratorFramework framework =
CuratorFrameworkFactory.builder()
.connectString(curatorConfig.getZkHosts())
.retryPolicy(
new ExponentialBackoffRetry(
1000,
30
)
)
.sessionTimeoutMs(curatorConfig.getZkSessionTimeoutMs())
.retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30))
// Don't compress stuff written just yet, need to get code deployed first.
.compressionProvider(new PotentiallyGzippedCompressionProvider(false))
.build();
lifecycle.addHandler(
@ -373,11 +310,6 @@ public class Initialization
);
}
public static String makePropPath(String basePath)
{
return String.format("%s/%s", basePath, PROP_SUBPATH);
}
public static String addressFromHost(final String host)
{
final int colon = host.indexOf(':');
@ -387,104 +319,4 @@ public class Initialization
return host.substring(0, colon);
}
}
/**
* Validate and Resolve Properties.
* Resolve zpaths with props like druid.zk.paths.*Path using druid.zk.paths.base value.
* Check validity so that if druid.zk.paths.*Path props are set, all are set,
* if none set, then construct defaults relative to druid.zk.paths.base and add these
* to the properties chain.
*
* @param props
*
* @return true if valid zpath properties.
*/
public static boolean validateResolveProps(Properties props)
{
boolean zpathValidateFailed;// validate druid.zk.paths.base
String propertyZpath = props.getProperty("druid.zk.paths.base");
zpathValidateFailed = zpathBaseCheck(propertyZpath, "property druid.zk.paths.base");
String zpathEffective = DEFAULT_ZPATH;
if (propertyZpath != null) {
zpathEffective = propertyZpath;
}
final String propertiesZpathOverride = props.getProperty("druid.zk.paths.propertiesPath");
if (!zpathValidateFailed) {
System.out.println("Effective zpath prefix=" + zpathEffective);
}
// validate druid.zk.paths.*Path properties
//
// if any zpath overrides are set in properties, they must start with /
int zpathOverrideCount = 0;
StringBuilder sbErrors = new StringBuilder(100);
for (int i = 0; i < SUB_PATH_PROPS.length; i++) {
String val = props.getProperty(SUB_PATH_PROPS[i]);
if (val != null) {
zpathOverrideCount++;
if (!val.startsWith("/")) {
sbErrors.append(SUB_PATH_PROPS[i]).append("=").append(val).append("\n");
zpathValidateFailed = true;
}
}
}
// separately check druid.zk.paths.propertiesPath (not in SUB_PATH_PROPS since it is not a "dir")
if (propertiesZpathOverride != null) {
zpathOverrideCount++;
if (!propertiesZpathOverride.startsWith("/")) {
sbErrors.append("druid.zk.paths.propertiesPath").append("=").append(propertiesZpathOverride).append("\n");
zpathValidateFailed = true;
}
}
if (zpathOverrideCount == 0) {
if (propertyZpath == null) { // if default base is used, store it as documentation
props.setProperty("druid.zk.paths.base", zpathEffective);
}
//
// Resolve default zpaths using zpathEffective as base
//
for (int i = 0; i < SUB_PATH_PROPS.length; i++) {
props.setProperty(SUB_PATH_PROPS[i], zpathEffective + "/" + SUB_PATHS[i]);
}
props.setProperty("druid.zk.paths.propertiesPath", zpathEffective + "/properties");
}
if (zpathValidateFailed) {
System.err.println(
"When overriding zk zpaths, with properties like druid.zk.paths.*Path " +
"the znode path must start with '/' (slash) ; problem overrides:"
);
System.err.print(sbErrors.toString());
}
return !zpathValidateFailed;
}
/**
* Check znode zpath base for proper slash, no trailing slash.
*
* @param zpathBase znode base path, if null then this method does nothing.
* @param errorMsgPrefix error context to use if errors are emitted, should indicate
* where the zpathBase value came from.
*
* @return true if validate failed.
*/
public static boolean zpathBaseCheck(String zpathBase, String errorMsgPrefix)
{
boolean zpathValidateFailed = false;
if (zpathBase != null) {
if (!zpathBase.startsWith("/")) {
zpathValidateFailed = true;
System.err.println(errorMsgPrefix + " must start with '/' (slash); found=" + zpathBase);
}
if (zpathBase.endsWith("/")) {
zpathValidateFailed = true;
System.err.println(errorMsgPrefix + " must NOT end with '/' (slash); found=" + zpathBase);
}
}
return zpathValidateFailed;
}
}

View File

@ -0,0 +1,62 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.initialization;
import org.skife.config.Config;
public abstract class ZkPathsConfig
{
@Config("druid.zk.paths.base")
public abstract String getZkBasePath();
@Config("druid.zk.paths.propertiesPath")
public String getPropertiesPath()
{
return defaultPath("properties");
}
@Config("druid.zk.paths.announcementsPath")
public String getAnnouncementsPath()
{
return defaultPath("announcements");
}
@Config("druid.zk.paths.servedSegmentsPath")
public String getServedSegmentsPath()
{
return defaultPath("servedSegments");
}
@Config("druid.zk.paths.loadQueuePath")
public String getLoadQueuePath()
{
return defaultPath("loadQueue");
}
@Config("druid.zk.paths.masterPath")
public String getMasterPath()
{
return defaultPath("master");
}
private String defaultPath(final String subPath) {
return String.format("%s/%s", getZkBasePath(), subPath);
}
}

View File

@ -1,68 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.zk;
import com.metamx.common.IAE;
import org.I0Itec.zkclient.exception.ZkMarshallingError;
import org.I0Itec.zkclient.serialize.ZkSerializer;
import java.io.*;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.Properties;
import java.util.TimeZone;
/**
*/
public class PropertiesZkSerializer implements ZkSerializer
{
private static final SimpleDateFormat df = new SimpleDateFormat("yyyyMMdd'T'HHmmss'z'");
static {
df.setTimeZone(TimeZone.getTimeZone("UTC"));
}
public final static String META_PROP = "__MODIFIED";
@Override
public byte[] serialize(Object data) throws ZkMarshallingError
{
if (data instanceof Properties) {
final Properties props = (Properties) data;
ByteArrayOutputStream bos = new ByteArrayOutputStream(props.size()*60 + 30);
try {
final String ts = df.format(new Date());
props.setProperty("__MODIFIED", ts);
props.store(bos, "Druid");
} catch (IOException ignored) { }
return bos.toByteArray();
}
throw new IAE("Can only serialize Properties into ZK");
}
@Override
public Object deserialize(byte[] bytes) throws ZkMarshallingError
{
final Properties props = new Properties();
try {
props.load(new ByteArrayInputStream(bytes));
} catch (IOException ignored) {
}
return props;
}
}

View File

@ -1,198 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.phonebook;
import com.metamx.common.ISE;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import java.util.List;
import java.util.Map;
/**
*/
public class BasePhoneBook implements PhoneBook
{
private final Object lock = new Object();
private final StoppedPhoneBook stoppedPages = new StoppedPhoneBook();
private final PhoneBook actualPages;
private volatile boolean started = false;
public BasePhoneBook(PhoneBook actualPages)
{
this.actualPages = actualPages;
}
@Override
@LifecycleStart
public void start()
{
synchronized (lock) {
actualPages.start();
for (Map.Entry<String, Map<String, Object>> services : stoppedPages.getAnnouncements().entrySet()) {
String serviceName = services.getKey();
for (Map.Entry<String, Object> announcements : services.getValue().entrySet()) {
String nodeName = announcements.getKey();
actualPages.announce(serviceName, nodeName, announcements.getValue());
}
}
for (Map.Entry<String, PhoneBookPeon> listenerEntry : stoppedPages.getListeners().entries()) {
actualPages.registerListener(listenerEntry.getKey(), listenerEntry.getValue());
}
started = true;
}
}
@Override
@LifecycleStop
public void stop()
{
synchronized (lock) {
started = false;
for (Map.Entry<String, Map<String, Object>> services : stoppedPages.getAnnouncements().entrySet()) {
String serviceName = services.getKey();
for (String nodeName : services.getValue().keySet()) {
actualPages.unannounce(serviceName, nodeName);
}
}
for (Map.Entry<String, PhoneBookPeon> listenerEntry : stoppedPages.getListeners().entries()) {
actualPages.unregisterListener(listenerEntry.getKey(), listenerEntry.getValue());
}
actualPages.stop();
}
}
@Override
public boolean isStarted()
{
return started;
}
@Override
public <T> void announce(String serviceName, String nodeName, T properties)
{
synchronized (lock) {
stoppedPages.announce(serviceName, nodeName, properties);
if (started) {
actualPages.announce(serviceName, nodeName, properties);
}
}
}
@Override
public void unannounce(String serviceName, String nodeName)
{
synchronized (lock) {
stoppedPages.unannounce(serviceName, nodeName);
if (started) {
actualPages.unannounce(serviceName, nodeName);
}
}
}
@Override
public <T> T lookup(String serviceName, Class<? extends T> clazz)
{
synchronized (lock) {
if (! started) {
throw new ISE("Cannot lookup on a stopped PhoneBook.");
}
return actualPages.lookup(serviceName, clazz);
}
}
@Override
public <T> void post(String serviceName, String nodeName, T properties)
{
synchronized (lock) {
if (! started) {
throw new ISE("Cannot post to a stopped PhoneBook.");
}
actualPages.post(serviceName, nodeName, properties);
}
}
@Override
public boolean unpost(String serviceName, String nodeName)
{
synchronized (lock) {
if (! started) {
throw new ISE("Cannot post to a stopped PhoneBook.");
}
return actualPages.unpost(serviceName, nodeName);
}
}
@Override
public <T> void postEphemeral(String serviceName, String nodeName, T properties)
{
synchronized (lock) {
if (! started) {
throw new ISE("Cannot post to a stopped PhoneBook.");
}
actualPages.postEphemeral(serviceName, nodeName, properties);
}
}
@Override
public <T> void registerListener(String serviceName, PhoneBookPeon<T> peon)
{
synchronized (lock) {
stoppedPages.registerListener(serviceName, peon);
if (started) {
actualPages.registerListener(serviceName, peon);
}
}
}
@Override
public <T> void unregisterListener(String serviceName, PhoneBookPeon<T> peon)
{
synchronized (lock) {
stoppedPages.unregisterListener(serviceName, peon);
if (started) {
actualPages.unregisterListener(serviceName, peon);
}
}
}
@Override
public String combineParts(List<String> parts)
{
return actualPages.combineParts(parts);
}
}

View File

@ -1,133 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.phonebook;
import org.apache.commons.codec.digest.DigestUtils;
import java.math.BigInteger;
import java.util.Map;
import java.util.TreeMap;
/**
*/
public class ConsistentHashLookup implements ServiceLookup
{
private final TreeMap<BigInteger, Map<String, String>> hashCircle = new TreeMap<BigInteger, Map<String, String>>();
private final PhoneBook yp;
private final int numVirtualNodes;
private final Hasher hasher;
public ConsistentHashLookup(
PhoneBook yp,
String serviceName
)
{
this(yp, serviceName, 100);
}
public ConsistentHashLookup(
PhoneBook yp,
String serviceName,
final int numVirtualNodes
)
{
this.yp = yp;
this.numVirtualNodes = numVirtualNodes;
this.hasher = new Hasher();
yp.registerListener(
serviceName,
new RingUpdaterPeon(
hashCircle,
numVirtualNodes,
hasher
)
);
}
@Override
public Map<String, String> get(String lookupKey)
{
if (hashCircle.isEmpty()) {
return null;
}
BigInteger key = hasher.hash(lookupKey);
Map.Entry<BigInteger, Map<String, String>> retEntry = hashCircle.ceilingEntry(key);
if (retEntry == null) {
retEntry = hashCircle.firstEntry();
}
return retEntry.getValue();
}
private static class Hasher
{
public BigInteger hash(String name)
{
return new BigInteger(DigestUtils.md5(name));
}
}
private static class RingUpdaterPeon implements PhoneBookPeon<Map>
{
private final TreeMap<BigInteger,Map<String,String>> hashCircle;
private final int numVirtualNodes;
private final Hasher hasher;
public RingUpdaterPeon(
TreeMap<BigInteger, Map<String, String>> hashCircle,
int numVirtualNodes,
Hasher hasher
)
{
this.hashCircle = hashCircle;
this.numVirtualNodes = numVirtualNodes;
this.hasher = hasher;
}
@Override
public Class<Map> getObjectClazz()
{
return Map.class;
}
@Override
public void newEntry(String name, Map properties)
{
for (int i = 0; i < numVirtualNodes; i++) {
hashCircle.put(hasher.hash(name + i), properties);
}
}
@Override
public void entryRemoved(String name)
{
for (int i = 0; i < numVirtualNodes; i++) {
hashCircle.remove(hasher.hash(name + i));
}
}
}
}

View File

@ -1,55 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.phonebook;
import java.util.List;
/**
* A PhoneBook object is just like a phone book. You can publish ("announce") your services to it as well as
* find out about other people publishing their services (registerListener).
*
* Finding out about other people's announcements is accomplished by employing a Peon, who gets notified of
* announcements coming and going and does something with them.
*/
public interface PhoneBook
{
public void start();
public void stop();
public boolean isStarted();
public <T> void announce(String serviceName, String nodeName, T properties);
public void unannounce(String serviceName, String nodeName);
public <T> T lookup(String serviceName, Class<? extends T> clazz);
public <T> void post(String serviceName, String nodeName, T properties);
public boolean unpost(String serviceName, String nodeName);
public <T> void postEphemeral(String serviceName, String nodeName, T properties);
public <T> void registerListener(String serviceName, PhoneBookPeon<T> peon);
public <T> void unregisterListener(String serviceName, PhoneBookPeon<T> peon);
/**
* A method to combine a number of hierarchical parts into a String that would "work" for this PhoneBook implementation.
*
* I.e., a call to combineParts("A", "B") should return the String "serviceName" that can be used to register a
* listener underneath something that was announced via a call to announce("A", "B", {})
*
* @param parts
* @return
*/
public String combineParts(List<String> parts);
}

View File

@ -1,32 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.phonebook;
/**
* A PhoneBookPeon is a Dilbert-like character who sits in his cubicle all day just waiting to hear about newEntry()s
* and removed entries. He acts as the go between, someone that the PhoneBook knows how to talk to and can then
* translate said message into whatever his employer wants.
*/
public interface PhoneBookPeon<T>
{
public Class<T> getObjectClazz();
public void newEntry(String name, T properties);
public void entryRemoved(String name);
}

View File

@ -1,32 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.phonebook;
import java.util.Map;
/**
* A ServiceLookup is an object that, when given a key, will return a metadata map for that key. This was created
* for use in doing things like consistent hashing, where the lookupKey represents the partition key and the
* metadata map has stuff like host and port in it (basically, the information required to be able to contact the server)
*/
public interface ServiceLookup
{
public Map<String, String> get(String lookupKey);
}

View File

@ -1,175 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.phonebook;
import com.google.common.base.Supplier;
import com.google.common.collect.Constraint;
import com.google.common.collect.Constraints;
import com.google.common.collect.Maps;
import com.google.common.collect.Multimap;
import com.google.common.collect.Multimaps;
import com.google.common.collect.Sets;
import com.metamx.common.IAE;
import com.metamx.common.logger.Logger;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* A class that collects announcements for you. Can be used to simplify start/stop logic
*
* Not thread-safe
*/
class StoppedPhoneBook implements PhoneBook
{
private static final Logger log = new Logger(StoppedPhoneBook.class);
private final Map<String, Map<String, Object>> announcements = Maps.newHashMap();
private final Multimap<String, PhoneBookPeon> listeners = Multimaps.newSetMultimap(
Maps.<String, Collection<PhoneBookPeon>>newHashMap(),
new Supplier<Set<PhoneBookPeon>>()
{
@Override
public Set<PhoneBookPeon> get()
{
final HashSet<PhoneBookPeon> theSet = Sets.newHashSet();
return Constraints.constrainedSet(
theSet,
new Constraint<PhoneBookPeon>()
{
@Override
public PhoneBookPeon checkElement(PhoneBookPeon element)
{
if (theSet.contains(element)) {
throw new IAE("Listener[%s] has already been registered", element);
}
return element;
}
});
}
}
);
@Override
public void start()
{
throw new UnsupportedOperationException();
}
@Override
public void stop()
{
throw new UnsupportedOperationException();
}
@Override
public boolean isStarted()
{
return false;
}
@Override
public <T> void announce(String serviceName, String nodeName, T properties)
{
Map<String, Object> serviceAnnouncements = announcements.get(serviceName);
if (serviceAnnouncements == null) {
serviceAnnouncements = Maps.newHashMap();
announcements.put(serviceName, serviceAnnouncements);
}
serviceAnnouncements.put(nodeName, properties);
}
@Override
public void unannounce(String serviceName, String nodeName)
{
Map<String, Object> serviceAnnouncements = announcements.get(serviceName);
if (serviceAnnouncements == null) {
throw new IAE("Cannot unannounce[%s]: No announcements for service[%s]", nodeName, serviceName);
}
if (! serviceAnnouncements.containsKey(nodeName)) {
log.warn("Cannot unannounce[%s]: it doesn't exist for service[%s]", nodeName, serviceName);
return;
}
serviceAnnouncements.remove(nodeName);
}
@Override
public <T> T lookup(String serviceName, Class<? extends T> clazz)
{
throw new UnsupportedOperationException();
}
@Override
public <T> void post(String serviceName, String nodeName, T properties)
{
throw new UnsupportedOperationException();
}
@Override
public boolean unpost(String serviceName, String nodeName)
{
throw new UnsupportedOperationException();
}
@Override
public <T> void postEphemeral(String serviceName, String nodeName, T properties)
{
throw new UnsupportedOperationException();
}
@Override
public <T> void registerListener(String serviceName, PhoneBookPeon<T> peon)
{
listeners.put(serviceName, peon);
}
@Override
public <T> void unregisterListener(String serviceName, PhoneBookPeon<T> peon)
{
if (! listeners.remove(serviceName, peon)) {
throw new IAE("Cannot unregister listener[%s] on service[%s] that wasn't first registered.", serviceName, peon);
}
}
@Override
public String combineParts(List<String> parts)
{
throw new UnsupportedOperationException("This should never be called");
}
public Map<String, Map<String, Object>> getAnnouncements()
{
return announcements;
}
public Multimap<String, PhoneBookPeon> getListeners()
{
return listeners;
}
}

View File

@ -0,0 +1,38 @@
package com.metamx.druid.curator;
import com.google.common.io.Closeables;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.RetryOneTime;
import org.apache.curator.test.TestingServer;
import org.apache.curator.test.Timing;
/**
*/
public class CuratorTestBase
{
protected TestingServer server;
protected Timing timing;
protected CuratorFramework curator;
protected void setupServerAndCurator() throws Exception
{
server = new TestingServer();
timing = new Timing();
curator = CuratorFrameworkFactory
.builder()
.connectString(server.getConnectString())
.sessionTimeoutMs(timing.session())
.connectionTimeoutMs(timing.connection())
.retryPolicy(new RetryOneTime(1))
.compressionProvider(new PotentiallyGzippedCompressionProvider(true))
.build();
}
protected void tearDownServerAndCurator()
{
Closeables.closeQuietly(curator);
Closeables.closeQuietly(server);
}
}

View File

@ -0,0 +1,166 @@
/*
* 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 com.metamx.druid.curator.announcement;
import com.google.common.collect.Sets;
import com.metamx.druid.concurrent.Execs;
import com.metamx.druid.curator.CuratorTestBase;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.CuratorEvent;
import org.apache.curator.framework.api.CuratorEventType;
import org.apache.curator.framework.api.CuratorListener;
import org.apache.curator.test.KillSession;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
/**
*/
public class AnnouncerTest extends CuratorTestBase
{
private ExecutorService exec;
@Before
public void setUp() throws Exception
{
setupServerAndCurator();
exec = Execs.singleThreaded("test-announcer-sanity-%s");
}
@After
public void tearDown()
{
tearDownServerAndCurator();
}
@Test
public void testSanity() throws Exception
{
curator.start();
curator.create().forPath("/somewhere");
Announcer announcer = new Announcer(curator, exec);
final byte[] billy = "billy".getBytes();
final String testPath1 = "/test1";
final String testPath2 = "/somewhere/test2";
announcer.announce(testPath1, billy);
Assert.assertNull(curator.checkExists().forPath(testPath1));
Assert.assertNull(curator.checkExists().forPath(testPath2));
announcer.start();
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
Assert.assertNull(curator.checkExists().forPath(testPath2));
announcer.announce(testPath2, billy);
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
final CountDownLatch latch = new CountDownLatch(1);
curator.getCuratorListenable().addListener(
new CuratorListener()
{
@Override
public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
{
if (event.getType() == CuratorEventType.CREATE && event.getPath().equals(testPath1)) {
latch.countDown();
}
}
}
);
curator.delete().forPath(testPath1);
Assert.assertTrue(timing.awaitLatch(latch));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
announcer.unannounce(testPath1);
Assert.assertNull(curator.checkExists().forPath(testPath1));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
announcer.stop();
Assert.assertNull(curator.checkExists().forPath(testPath1));
Assert.assertNull(curator.checkExists().forPath(testPath2));
}
@Test
public void testSessionKilled() throws Exception
{
final ExecutorService exec = Execs.singleThreaded("test-announcer-sanity-%s");
curator.start();
Announcer announcer = new Announcer(curator, exec);
try {
curator.create().forPath("/somewhere");
announcer.start();
final byte[] billy = "billy".getBytes();
final String testPath1 = "/test1";
final String testPath2 = "/somewhere/test2";
final Set<String> paths = Sets.newHashSet(testPath1, testPath2);
announcer.announce(testPath1, billy);
announcer.announce(testPath2, billy);
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
final CountDownLatch latch = new CountDownLatch(1);
curator.getCuratorListenable().addListener(
new CuratorListener()
{
@Override
public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
{
if (event.getType() == CuratorEventType.CREATE) {
paths.remove(event.getPath());
if (paths.isEmpty()) {
latch.countDown();
}
}
}
}
);
KillSession.kill(curator.getZookeeperClient().getZooKeeper(), server.getConnectString());
Assert.assertTrue(timing.awaitLatch(latch));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
announcer.stop();
Assert.assertNull(curator.checkExists().forPath(testPath1));
Assert.assertNull(curator.checkExists().forPath(testPath2));
}
finally {
announcer.stop();
}
}
}

View File

@ -0,0 +1,245 @@
package com.metamx.druid.curator.inventory;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import com.metamx.druid.concurrent.Execs;
import com.metamx.druid.curator.CuratorTestBase;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.CuratorEvent;
import org.apache.curator.framework.api.CuratorEventType;
import org.apache.curator.framework.api.CuratorListener;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.Watcher;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
/**
*/
public class CuratorInventoryManagerTest extends CuratorTestBase
{
private ExecutorService exec;
@Before
public void setUp() throws Exception
{
setupServerAndCurator();
exec = Execs.singleThreaded("curator-inventory-manager-test-%s");
}
@Test
public void testSanity() throws Exception
{
final MapStrategy strategy = new MapStrategy();
CuratorInventoryManager<Map<String, Integer>, Integer> manager = new CuratorInventoryManager<Map<String, Integer>, Integer>(
curator, new StringInventoryManagerConfig("/container", "/inventory"), exec, strategy
);
curator.start();
manager.start();
curator.create().creatingParentsIfNeeded().forPath("/container");
curator.create().creatingParentsIfNeeded().forPath("/inventory/billy");
Assert.assertTrue(Iterables.isEmpty(manager.getInventory()));
CountDownLatch containerLatch = new CountDownLatch(1);
strategy.setNewContainerLatch(containerLatch);
curator.create().withMode(CreateMode.EPHEMERAL).forPath("/container/billy", new byte[]{});
Assert.assertTrue(timing.awaitLatch(containerLatch));
strategy.setNewContainerLatch(null);
final Iterable<Map<String, Integer>> inventory = manager.getInventory();
Assert.assertTrue(Iterables.getOnlyElement(inventory).isEmpty());
CountDownLatch inventoryLatch = new CountDownLatch(2);
strategy.setNewInventoryLatch(inventoryLatch);
curator.create().withMode(CreateMode.EPHEMERAL).forPath("/inventory/billy/1", Ints.toByteArray(100));
curator.create().withMode(CreateMode.EPHEMERAL).forPath("/inventory/billy/bob", Ints.toByteArray(2287));
Assert.assertTrue(timing.awaitLatch(inventoryLatch));
strategy.setNewInventoryLatch(null);
verifyInventory(manager);
CountDownLatch deleteLatch = new CountDownLatch(1);
strategy.setDeadInventoryLatch(deleteLatch);
curator.delete().forPath("/inventory/billy/1");
Assert.assertTrue(timing.awaitLatch(deleteLatch));
strategy.setDeadInventoryLatch(null);
Assert.assertEquals(1, manager.getInventoryValue("billy").size());
Assert.assertEquals(2287, manager.getInventoryValue("billy").get("bob").intValue());
inventoryLatch = new CountDownLatch(1);
strategy.setNewInventoryLatch(inventoryLatch);
curator.create().withMode(CreateMode.EPHEMERAL).forPath("/inventory/billy/1", Ints.toByteArray(100));
Assert.assertTrue(timing.awaitLatch(inventoryLatch));
strategy.setNewInventoryLatch(null);
verifyInventory(manager);
final CountDownLatch latch = new CountDownLatch(1);
curator.getCuratorListenable().addListener(
new CuratorListener()
{
@Override
public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
{
if (event.getType() == CuratorEventType.WATCHED
&& event.getWatchedEvent().getState() == Watcher.Event.KeeperState.Disconnected)
{
latch.countDown();
}
}
}
);
server.stop();
Assert.assertTrue(timing.awaitLatch(latch));
verifyInventory(manager);
Thread.sleep(50); // Wait a bit
verifyInventory(manager);
}
private void verifyInventory(CuratorInventoryManager<Map<String, Integer>, Integer> manager)
{
final Map<String, Integer> vals = manager.getInventoryValue("billy");
Assert.assertEquals(2, vals.size());
Assert.assertEquals(100, vals.get("1").intValue());
Assert.assertEquals(2287, vals.get("bob").intValue());
}
private static class StringInventoryManagerConfig implements InventoryManagerConfig
{
private final String containerPath;
private final String inventoryPath;
private StringInventoryManagerConfig(
String containerPath,
String inventoryPath
)
{
this.containerPath = containerPath;
this.inventoryPath = inventoryPath;
}
@Override
public String getContainerPath()
{
return containerPath;
}
@Override
public String getInventoryPath()
{
return inventoryPath;
}
}
private static class MapStrategy implements CuratorInventoryManagerStrategy<Map<String, Integer>, Integer>
{
private volatile CountDownLatch newContainerLatch = null;
private volatile CountDownLatch deadContainerLatch = null;
private volatile CountDownLatch newInventoryLatch = null;
private volatile CountDownLatch deadInventoryLatch = null;
@Override
public Map<String, Integer> deserializeContainer(byte[] bytes)
{
return Maps.newTreeMap();
}
@Override
public byte[] serializeContainer(Map<String, Integer> container)
{
return new byte[]{};
}
@Override
public Integer deserializeInventory(byte[] bytes)
{
return Ints.fromByteArray(bytes);
}
@Override
public byte[] serializeInventory(Integer inventory)
{
return Ints.toByteArray(inventory);
}
@Override
public void newContainer(Map<String, Integer> newContainer)
{
if (newContainerLatch != null) {
newContainerLatch.countDown();
}
}
@Override
public void deadContainer(Map<String, Integer> deadContainer)
{
if (deadContainerLatch != null) {
deadContainerLatch.countDown();
}
}
@Override
public Map<String, Integer> updateContainer(Map<String, Integer> oldContainer, Map<String, Integer> newContainer)
{
newContainer.putAll(oldContainer);
return newContainer;
}
@Override
public Map<String, Integer> addInventory(Map<String, Integer> container, String inventoryKey, Integer inventory)
{
container.put(inventoryKey, inventory);
if (newInventoryLatch != null) {
newInventoryLatch.countDown();
}
return container;
}
@Override
public Map<String, Integer> removeInventory(Map<String, Integer> container, String inventoryKey)
{
container.remove(inventoryKey);
if (deadInventoryLatch != null) {
deadInventoryLatch.countDown();
}
return container;
}
private void setNewContainerLatch(CountDownLatch newContainerLatch)
{
this.newContainerLatch = newContainerLatch;
}
private void setDeadContainerLatch(CountDownLatch deadContainerLatch)
{
this.deadContainerLatch = deadContainerLatch;
}
private void setNewInventoryLatch(CountDownLatch newInventoryLatch)
{
this.newInventoryLatch = newInventoryLatch;
}
private void setDeadInventoryLatch(CountDownLatch deadInventoryLatch)
{
this.deadInventoryLatch = deadInventoryLatch;
}
}
}

View File

@ -56,11 +56,11 @@
<artifactId>config-magic</artifactId>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-recipes</artifactId>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-x-discovery</artifactId>
</dependency>
<dependency>

View File

@ -0,0 +1,37 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.concurrent;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
/**
*/
public class Execs
{
public static ExecutorService singleThreaded(String nameFormat)
{
return Executors.newSingleThreadExecutor(
new ThreadFactoryBuilder().setDaemon(true).setNameFormat(nameFormat).build()
);
}
}

View File

@ -122,10 +122,6 @@
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</dependency>
<dependency>
<groupId>com.github.sgroschupf</groupId>
<artifactId>zkclient</artifactId>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -4,14 +4,11 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.ZKPhoneBook;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.coordination.DataSegmentAnnouncer;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.realtime.RealtimeNode;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.SegmentPublisher;
import com.metamx.phonebook.PhoneBook;
import java.io.File;
import java.io.IOException;
@ -33,17 +30,8 @@ public class RealtimeStandaloneMain
RealtimeNode rn = RealtimeNode.builder().build();
lifecycle.addManagedInstance(rn);
// force standalone demo behavior (no zk, no db, no master, no broker)
//
// dummyPhoneBook will not be start()ed so it will not hang connecting to a nonexistent zk
PhoneBook dummyPhoneBook = new ZKPhoneBook(new DefaultObjectMapper(), null, null) {
@Override
public boolean isStarted() { return true;}
};
rn.setPhoneBook(dummyPhoneBook);
SegmentAnnouncer dummySegmentAnnouncer =
new SegmentAnnouncer()
DataSegmentAnnouncer dummySegmentAnnouncer =
new DataSegmentAnnouncer()
{
@Override
public void announceSegment(DataSegment segment) throws IOException
@ -68,7 +56,7 @@ public class RealtimeStandaloneMain
};
// dummySegmentPublisher will not send updates to db because standalone demo has no db
rn.setSegmentAnnouncer(dummySegmentAnnouncer);
rn.setAnnouncer(dummySegmentAnnouncer);
rn.setSegmentPublisher(dummySegmentPublisher);
rn.setDataSegmentPusher(
new DataSegmentPusher()

View File

@ -137,10 +137,6 @@
<artifactId>twitter4j-stream</artifactId>
<version>2.2.6</version>
</dependency>
<dependency>
<groupId>com.github.sgroschupf</groupId>
<artifactId>zkclient</artifactId>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -4,14 +4,11 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.ZKPhoneBook;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.coordination.DataSegmentAnnouncer;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.realtime.RealtimeNode;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.SegmentPublisher;
import com.metamx.phonebook.PhoneBook;
import druid.examples.twitter.TwitterSpritzerFirehoseFactory;
import java.io.File;
@ -35,17 +32,8 @@ public class RealtimeStandaloneMain
// register the Firehose
rn.registerJacksonSubtype(new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"));
// force standalone demo behavior (no zk, no db, no master, no broker)
//
// dummyPhoneBook will not be start()ed so it will not hang connecting to a nonexistent zk
PhoneBook dummyPhoneBook = new ZKPhoneBook(new DefaultObjectMapper(), null, null) {
@Override
public boolean isStarted() { return true;}
};
rn.setPhoneBook(dummyPhoneBook);
final SegmentAnnouncer dummySegmentAnnouncer =
new SegmentAnnouncer()
final DataSegmentAnnouncer dummySegmentAnnouncer =
new DataSegmentAnnouncer()
{
@Override
public void announceSegment(DataSegment segment) throws IOException
@ -70,8 +58,8 @@ public class RealtimeStandaloneMain
};
// dummySegmentPublisher will not send updates to db because standalone demo has no db
rn.setSegmentAnnouncer(dummySegmentAnnouncer);
rn.setSegmentPublisher(dummySegmentPublisher);
rn.setAnnouncer(dummySegmentAnnouncer);
rn.setDataSegmentPusher(
new DataSegmentPusher()
{

View File

@ -75,6 +75,6 @@ public class ComplexColumnPartSerde implements ColumnPartSerde
@Override
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
{
return serde.deserializeColumn(buffer, builder);
return serde == null ? this : serde.deserializeColumn(buffer, builder);
}
}

View File

@ -98,10 +98,6 @@
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</dependency>
<dependency>
<groupId>com.github.sgroschupf</groupId>
<artifactId>zkclient</artifactId>
</dependency>
<dependency>
<groupId>net.java.dev.jets3t</groupId>
<artifactId>jets3t</artifactId>

View File

@ -93,11 +93,11 @@
<artifactId>config-magic</artifactId>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-framework</artifactId>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-recipes</artifactId>
</dependency>
<dependency>
@ -156,10 +156,6 @@
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</dependency>
<dependency>
<groupId>com.github.sgroschupf</groupId>
<artifactId>zkclient</artifactId>
</dependency>
<!-- Tests -->
<dependency>
@ -173,8 +169,9 @@
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-test</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View File

@ -22,11 +22,12 @@ package com.metamx.druid.merger.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Maps;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.client.ServerView;
import com.metamx.druid.coordination.DataSegmentAnnouncer;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.MMappedQueryableIndexFactory;
import com.metamx.druid.loading.S3DataSegmentPuller;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.SegmentLoaderConfig;
import com.metamx.druid.loading.SegmentLoadingException;
import com.metamx.druid.loading.SingleSegmentLoader;
@ -35,7 +36,6 @@ import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.emitter.service.ServiceEmitter;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
@ -55,8 +55,8 @@ public class TaskToolbox
private final RestS3Service s3Client;
private final DataSegmentPusher segmentPusher;
private final DataSegmentKiller dataSegmentKiller;
private final SegmentAnnouncer segmentAnnouncer;
private final MutableServerView newSegmentServerView;
private final DataSegmentAnnouncer segmentAnnouncer;
private final ServerView newSegmentServerView;
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
private final ObjectMapper objectMapper;
@ -68,8 +68,8 @@ public class TaskToolbox
RestS3Service s3Client,
DataSegmentPusher segmentPusher,
DataSegmentKiller dataSegmentKiller,
SegmentAnnouncer segmentAnnouncer,
MutableServerView newSegmentServerView,
DataSegmentAnnouncer segmentAnnouncer,
ServerView newSegmentServerView,
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
ObjectMapper objectMapper
)
@ -112,12 +112,12 @@ public class TaskToolbox
return dataSegmentKiller;
}
public SegmentAnnouncer getSegmentAnnouncer()
public DataSegmentAnnouncer getSegmentAnnouncer()
{
return segmentAnnouncer;
}
public MutableServerView getNewSegmentServerView()
public ServerView getNewSegmentServerView()
{
return newSegmentServerView;
}

View File

@ -20,14 +20,14 @@
package com.metamx.druid.merger.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.client.ServerView;
import com.metamx.druid.coordination.DataSegmentAnnouncer;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.emitter.service.ServiceEmitter;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
@ -42,8 +42,8 @@ public class TaskToolboxFactory
private final RestS3Service s3Client;
private final DataSegmentPusher segmentPusher;
private final DataSegmentKiller dataSegmentKiller;
private final SegmentAnnouncer segmentAnnouncer;
private final MutableServerView newSegmentServerView;
private final DataSegmentAnnouncer segmentAnnouncer;
private final ServerView newSegmentServerView;
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
private final ObjectMapper objectMapper;
@ -54,8 +54,8 @@ public class TaskToolboxFactory
RestS3Service s3Client,
DataSegmentPusher segmentPusher,
DataSegmentKiller dataSegmentKiller,
SegmentAnnouncer segmentAnnouncer,
MutableServerView newSegmentServerView,
DataSegmentAnnouncer segmentAnnouncer,
ServerView newSegmentServerView,
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
ObjectMapper objectMapper
)
@ -72,11 +72,6 @@ public class TaskToolboxFactory
this.objectMapper = objectMapper;
}
public ObjectMapper getObjectMapper()
{
return objectMapper;
}
public TaskToolbox build(Task task)
{
return new TaskToolbox(

View File

@ -11,8 +11,8 @@ import com.metamx.druid.merger.common.task.Task;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.ToStringResponseHandler;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.netflix.curator.x.discovery.ServiceInstance;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.apache.curator.x.discovery.ServiceInstance;
import org.apache.curator.x.discovery.ServiceProvider;
import org.joda.time.Duration;
import java.io.IOException;

View File

@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.http.client.HttpClient;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.apache.curator.x.discovery.ServiceProvider;
/**
*/

View File

@ -75,6 +75,12 @@ public abstract class AbstractTask implements Task
return groupId;
}
@Override
public String getNodeType()
{
return null;
}
@JsonProperty
@Override
public String getDataSource()

View File

@ -47,7 +47,7 @@ import com.metamx.druid.realtime.MinTimeFirehose;
import com.metamx.druid.realtime.plumber.Plumber;
import com.metamx.druid.realtime.plumber.RealtimePlumberSchool;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.coordination.DataSegmentAnnouncer;
import com.metamx.druid.realtime.SegmentPublisher;
import com.metamx.druid.realtime.plumber.Sink;
import com.metamx.druid.realtime.plumber.VersioningPolicy;
@ -98,7 +98,7 @@ public class RealtimeIndexTask extends AbstractTask
@JsonProperty("id") String id,
@JsonProperty("schema") Schema schema,
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig, // TODO rename?
@JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig,
@JsonProperty("windowPeriod") Period windowPeriod,
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity,
@JsonProperty("minTime") DateTime minTime
@ -131,6 +131,12 @@ public class RealtimeIndexTask extends AbstractTask
return "index_realtime";
}
@Override
public String getNodeType()
{
return "realtime";
}
@Override
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
{
@ -178,8 +184,8 @@ public class RealtimeIndexTask extends AbstractTask
firehose = new GracefulShutdownFirehose(wrappedFirehose, segmentGranularity, windowPeriod);
}
// TODO -- Take PlumberSchool in constructor (although that will need jackson injectables for stuff like
// TODO -- the ServerView, which seems kind of odd?)
// It would be nice to get the PlumberSchool in the constructor. Although that will need jackson injectables for
// stuff like the ServerView, which seems kind of odd? Perhaps revisit this when Guice has been introduced.
final RealtimePlumberSchool realtimePlumberSchool = new RealtimePlumberSchool(
windowPeriod,
new File(toolbox.getTaskWorkDir(), "persist"),
@ -188,13 +194,12 @@ public class RealtimeIndexTask extends AbstractTask
final SegmentPublisher segmentPublisher = new TaskActionSegmentPublisher(this, toolbox);
// TODO -- We're adding stuff to talk to the coordinator in various places in the plumber, and may
// TODO -- want to be more robust to coordinator downtime (currently we'll block/throw in whatever
// TODO -- thread triggered the coordinator behavior, which will typically be either the main
// TODO -- data processing loop or the persist thread)
// NOTE: We talk to the coordinator in various places in the plumber and we could be more robust to issues
// with the coordinator. Right now, we'll block/throw in whatever thread triggered the coordinator behavior,
// which will typically be either the main data processing loop or the persist thread.
// Wrap default SegmentAnnouncer such that we unlock intervals as we unannounce segments
final SegmentAnnouncer lockingSegmentAnnouncer = new SegmentAnnouncer()
// Wrap default DataSegmentAnnouncer such that we unlock intervals as we unannounce segments
final DataSegmentAnnouncer lockingSegmentAnnouncer = new DataSegmentAnnouncer()
{
@Override
public void announceSegment(final DataSegment segment) throws IOException

View File

@ -76,6 +76,14 @@ public interface Task
*/
public String getType();
/**
* Get the nodeType for if/when this task publishes on zookeeper.
*
* @return the nodeType to use when publishing the server to zookeeper. null if the task doesn't expect to
* publish to zookeeper.
*/
public String getNodeType();
/**
* Returns the datasource this task operates on. Each task can operate on only one datasource.
*/

View File

@ -166,6 +166,11 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
}
}
String nodeType = task.getNodeType();
if (nodeType != null) {
command.add(String.format("-Ddruid.executor.nodeType=%s", nodeType));
}
command.add(String.format("-Ddruid.host=%s", childHost));
command.add(String.format("-Ddruid.port=%d", childPort));

View File

@ -49,11 +49,11 @@ import com.metamx.emitter.EmittingLogger;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.InputStreamResponseHandler;
import com.metamx.http.client.response.ToStringResponseHandler;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
import com.netflix.curator.utils.ZKPaths;
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.curator.utils.ZKPaths;
import org.apache.zookeeper.CreateMode;
import org.joda.time.DateTime;
import org.joda.time.Duration;

View File

@ -35,10 +35,10 @@ import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler;
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.recipes.leader.LeaderSelector;
import com.netflix.curator.framework.recipes.leader.LeaderSelectorListener;
import com.netflix.curator.framework.state.ConnectionState;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.leader.LeaderSelector;
import org.apache.curator.framework.recipes.leader.LeaderSelectorListener;
import org.apache.curator.framework.state.ConnectionState;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;

View File

@ -56,7 +56,7 @@ public interface TaskStorage
* Returns task as stored in the storage facility. If the task ID does not exist, this will return an
* absentee Optional.
*
* TODO -- This method probably wants to be combined with {@link #getStatus}.
* NOTE: This method really feels like it should be combined with {@link #getStatus}. Expect that in the future.
*/
public Optional<Task> getTask(String taskid);

View File

@ -143,12 +143,13 @@ public class TaskStorageQueryAdapter
* Returns all segments created by descendants for a particular task that stayed within the same task group. Includes
* that task, plus any tasks it spawned, and so on. Does not include spawned tasks that ended up in a different task
* group. Does not include this task's parents or siblings.
*
* This method is useful when you want to figure out all of the things a single task spawned. It does pose issues
* with the result set perhaps growing boundlessly and we do not do anything to protect against that. Use at your
* own risk and know that at some point, we might adjust this to actually enforce some sort of limits.
*/
public Set<DataSegment> getSameGroupNewSegments(final String taskid)
{
// TODO: This is useful for regular index tasks (so we know what was published), but
// TODO: for long-lived index tasks the list can get out of hand. We may want a limit.
final Optional<Task> taskOptional = storage.getTask(taskid);
final Set<DataSegment> segments = Sets.newHashSet();
final List<Task> nextTasks = Lists.newArrayList();

View File

@ -27,8 +27,8 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.worker.Worker;
import com.netflix.curator.framework.recipes.cache.ChildData;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.joda.time.DateTime;

View File

@ -22,10 +22,10 @@ package com.metamx.druid.merger.coordinator.http;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.services.ec2.AmazonEC2Client;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.base.Charsets;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.io.InputSupplier;
@ -41,7 +41,7 @@ import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.RegisteringNode;
import com.metamx.druid.QueryableNode;
import com.metamx.druid.config.ConfigManager;
import com.metamx.druid.config.ConfigManagerConfig;
import com.metamx.druid.config.JacksonConfigManager;
@ -107,8 +107,8 @@ import com.metamx.metrics.Monitor;
import com.metamx.metrics.MonitorScheduler;
import com.metamx.metrics.MonitorSchedulerConfig;
import com.metamx.metrics.SysMonitor;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.security.AWSCredentials;
@ -134,7 +134,7 @@ import java.util.concurrent.atomic.AtomicReference;
/**
*/
public class IndexerCoordinatorNode extends RegisteringNode
public class IndexerCoordinatorNode extends QueryableNode<IndexerCoordinatorNode>
{
private static final Logger log = new Logger(IndexerCoordinatorNode.class);
@ -143,11 +143,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
return new Builder();
}
private final Lifecycle lifecycle;
private final ObjectMapper jsonMapper;
private final Properties props;
private final ConfigurationObjectFactory configFactory;
private RestS3Service s3Service = null;
private List<Monitor> monitors = null;
private ServiceEmitter emitter = null;
@ -158,7 +153,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
private TaskStorage taskStorage = null;
private TaskQueue taskQueue = null;
private TaskLockbox taskLockbox = null;
private CuratorFramework curatorFramework = null;
private IndexerZkConfig indexerZkConfig;
private TaskRunnerFactory taskRunnerFactory = null;
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
@ -175,15 +169,11 @@ public class IndexerCoordinatorNode extends RegisteringNode
Properties props,
Lifecycle lifecycle,
ObjectMapper jsonMapper,
ObjectMapper smileMapper,
ConfigurationObjectFactory configFactory
)
{
super(ImmutableList.of(jsonMapper));
this.lifecycle = lifecycle;
this.props = props;
this.jsonMapper = jsonMapper;
this.configFactory = configFactory;
super("index-coordinator", log, props, lifecycle, jsonMapper, smileMapper, configFactory);
}
public IndexerCoordinatorNode setEmitter(ServiceEmitter emitter)
@ -222,12 +212,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
return this;
}
public IndexerCoordinatorNode setCuratorFramework(CuratorFramework curatorFramework)
{
this.curatorFramework = curatorFramework;
return this;
}
public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory)
{
this.taskRunnerFactory = taskRunnerFactory;
@ -248,14 +232,14 @@ public class IndexerCoordinatorNode extends RegisteringNode
public void doInit() throws Exception
{
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(getLifecycle());
initializeDB();
final ConfigManagerConfig managerConfig = configFactory.build(ConfigManagerConfig.class);
final ConfigManagerConfig managerConfig = getConfigFactory().build(ConfigManagerConfig.class);
DbConnector.createConfigTable(dbi, managerConfig.getConfigTable());
JacksonConfigManager configManager =
new JacksonConfigManager(
lifecycle.addManagedInstance(
getLifecycle().addManagedInstance(
new ConfigManager(
dbi,
managerConfig
@ -268,11 +252,10 @@ public class IndexerCoordinatorNode extends RegisteringNode
initializeMonitors();
initializeIndexerCoordinatorConfig();
initializeMergeDBCoordinator();
initializeJacksonSubtypes();
initializeTaskStorage();
initializeTaskLockbox();
initializeTaskQueue();
initializeJacksonSubtypes();
initializeCurator();
initializeIndexerZkConfig();
initializeTaskActionClientFactory();
initializeTaskRunnerFactory(configManager);
@ -284,12 +267,12 @@ public class IndexerCoordinatorNode extends RegisteringNode
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
final MonitorScheduler monitorScheduler = new MonitorScheduler(
configFactory.build(MonitorSchedulerConfig.class),
getConfigFactory().build(MonitorSchedulerConfig.class),
globalScheduledExec,
emitter,
monitors
);
lifecycle.addManagedInstance(monitorScheduler);
getLifecycle().addManagedInstance(monitorScheduler);
final Injector injector = Guice.createInjector(
new IndexerCoordinatorServletModule(
@ -314,8 +297,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
);
staticContext.setBaseResource(resourceCollection);
// TODO -- Need a QueryServlet and some kind of QuerySegmentWalker if we want to support querying tasks
// TODO -- (e.g. for realtime) in local mode
// If we want to support querying tasks (e.g. for realtime in local mode), we need a QueryServlet here.
final Context root = new Context(server, "/", Context.SESSIONS);
root.addServlet(new ServletHolder(new StatusServlet()), "/status");
@ -358,11 +340,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
initialized = true;
}
private ObjectMapper getJsonMapper()
{
return jsonMapper;
}
private void initializeTaskActionClientFactory()
{
if (taskActionClientFactory == null) {
@ -376,7 +353,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
private void initializeTaskMasterLifecycle()
{
if (taskMasterLifecycle == null) {
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
final ServiceDiscoveryConfig serviceDiscoveryConfig = getConfigFactory().build(ServiceDiscoveryConfig.class);
taskMasterLifecycle = new TaskMasterLifecycle(
taskQueue,
taskActionClientFactory,
@ -384,17 +361,17 @@ public class IndexerCoordinatorNode extends RegisteringNode
serviceDiscoveryConfig,
taskRunnerFactory,
resourceManagementSchedulerFactory,
curatorFramework,
getCuratorFramework(),
emitter
);
lifecycle.addManagedInstance(taskMasterLifecycle);
getLifecycle().addManagedInstance(taskMasterLifecycle);
}
}
private void initializePersistentTaskLogs() throws S3ServiceException
{
if (persistentTaskLogs == null) {
final TaskLogConfig taskLogConfig = configFactory.build(TaskLogConfig.class);
final TaskLogConfig taskLogConfig = getConfigFactory().build(TaskLogConfig.class);
if (taskLogConfig.getLogStorageBucket() != null) {
initializeS3Service();
persistentTaskLogs = new S3TaskLogs(
@ -444,21 +421,21 @@ public class IndexerCoordinatorNode extends RegisteringNode
doInit();
}
lifecycle.start();
getLifecycle().start();
}
@LifecycleStop
public synchronized void stop()
{
lifecycle.stop();
getLifecycle().stop();
}
private void initializeServer()
{
if (server == null) {
server = Initialization.makeJettyServer(configFactory.build(ServerConfig.class));
server = Initialization.makeJettyServer(getConfigFactory().build(ServerConfig.class));
lifecycle.addHandler(
getLifecycle().addHandler(
new Lifecycle.Handler()
{
@Override
@ -496,11 +473,11 @@ public class IndexerCoordinatorNode extends RegisteringNode
HttpClientConfig.builder().withNumConnections(1).withReadTimeout(
new Duration(
PropUtils.getProperty(
props,
getProps(),
"druid.emitter.timeOut"
)
)
).build(), lifecycle
).build(), getLifecycle()
);
}
}
@ -509,26 +486,14 @@ public class IndexerCoordinatorNode extends RegisteringNode
{
if (emitter == null) {
emitter = new ServiceEmitter(
PropUtils.getProperty(props, "druid.service"),
PropUtils.getProperty(props, "druid.host"),
Emitters.create(props, httpClient, getJsonMapper(), lifecycle)
PropUtils.getProperty(getProps(), "druid.service"),
PropUtils.getProperty(getProps(), "druid.host"),
Emitters.create(getProps(), httpClient, getJsonMapper(), getLifecycle())
);
}
EmittingLogger.registerEmitter(emitter);
}
private void initializeS3Service() throws S3ServiceException
{
if(s3Service == null) {
s3Service = new RestS3Service(
new AWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
)
);
}
}
private void initializeMonitors()
{
if (monitors == null) {
@ -541,7 +506,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
private void initializeDB()
{
if (dbConnectorConfig == null) {
dbConnectorConfig = configFactory.build(DbConnectorConfig.class);
dbConnectorConfig = getConfigFactory().build(DbConnectorConfig.class);
}
if (dbi == null) {
dbi = new DbConnector(dbConnectorConfig).getDBI();
@ -551,10 +516,20 @@ public class IndexerCoordinatorNode extends RegisteringNode
private void initializeIndexerCoordinatorConfig()
{
if (config == null) {
config = configFactory.build(IndexerCoordinatorConfig.class);
config = getConfigFactory().build(IndexerCoordinatorConfig.class);
}
}
public void initializeS3Service() throws S3ServiceException
{
this.s3Service = new RestS3Service(
new AWSCredentials(
PropUtils.getProperty(getProps(), "com.metamx.aws.accessKey"),
PropUtils.getProperty(getProps(), "com.metamx.aws.secretKey")
)
);
}
public void initializeMergeDBCoordinator()
{
if (mergerDBCoordinator == null) {
@ -581,21 +556,10 @@ public class IndexerCoordinatorNode extends RegisteringNode
}
}
public void initializeCurator() throws Exception
{
if (curatorFramework == null) {
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
curatorFramework = Initialization.makeCuratorFrameworkClient(
serviceDiscoveryConfig,
lifecycle
);
}
}
public void initializeIndexerZkConfig()
{
if (indexerZkConfig == null) {
indexerZkConfig = configFactory.build(IndexerZkConfig.class);
indexerZkConfig = getConfigFactory().build(IndexerZkConfig.class);
}
}
@ -605,7 +569,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
if (config.getStorageImpl().equals("local")) {
taskStorage = new HeapMemoryTaskStorage();
} else if (config.getStorageImpl().equals("db")) {
final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class);
final IndexerDbConnectorConfig dbConnectorConfig = getConfigFactory().build(IndexerDbConnectorConfig.class);
DbConnector.createTaskTable(dbi, dbConnectorConfig.getTaskTable());
DbConnector.createTaskLogTable(dbi, dbConnectorConfig.getTaskLogTable());
DbConnector.createTaskLockTable(dbi, dbConnectorConfig.getTaskLockTable());
@ -640,14 +604,15 @@ public class IndexerCoordinatorNode extends RegisteringNode
.build()
);
final CuratorFramework curator = getCuratorFramework();
return new RemoteTaskRunner(
getJsonMapper(),
configFactory.build(RemoteTaskRunnerConfig.class),
curatorFramework,
new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true),
getConfigFactory().build(RemoteTaskRunnerConfig.class),
curator,
new PathChildrenCache(curator, indexerZkConfig.getAnnouncementPath(), true),
retryScheduledExec,
new RetryPolicyFactory(
configFactory.buildWithReplacements(
getConfigFactory().buildWithReplacements(
RetryPolicyConfig.class,
ImmutableMap.of("base_path", "druid.indexing")
)
@ -666,8 +631,8 @@ public class IndexerCoordinatorNode extends RegisteringNode
{
final ExecutorService runnerExec = Executors.newFixedThreadPool(config.getNumLocalThreads());
return new ForkingTaskRunner(
configFactory.build(ForkingTaskRunnerConfig.class),
props,
getConfigFactory().build(ForkingTaskRunnerConfig.class),
getProps(),
persistentTaskLogs,
runnerExec,
getJsonMapper()
@ -705,11 +670,11 @@ public class IndexerCoordinatorNode extends RegisteringNode
getJsonMapper(),
new AmazonEC2Client(
new BasicAWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
PropUtils.getProperty(getProps(), "com.metamx.aws.accessKey"),
PropUtils.getProperty(getProps(), "com.metamx.aws.secretKey")
)
),
configFactory.build(EC2AutoScalingStrategyConfig.class),
getConfigFactory().build(EC2AutoScalingStrategyConfig.class),
workerSetupData
);
} else if (config.getStrategyImpl().equalsIgnoreCase("noop")) {
@ -722,10 +687,10 @@ public class IndexerCoordinatorNode extends RegisteringNode
runner,
new SimpleResourceManagementStrategy(
strategy,
configFactory.build(SimpleResourceManagmentConfig.class),
getConfigFactory().build(SimpleResourceManagmentConfig.class),
workerSetupData
),
configFactory.build(ResourceManagementSchedulerConfig.class),
getConfigFactory().build(ResourceManagementSchedulerConfig.class),
scalingScheduledExec
);
}
@ -767,8 +732,16 @@ public class IndexerCoordinatorNode extends RegisteringNode
public IndexerCoordinatorNode build()
{
if (jsonMapper == null) {
if (jsonMapper == null && smileMapper == null) {
jsonMapper = new DefaultObjectMapper();
smileMapper = new DefaultObjectMapper(new SmileFactory());
smileMapper.getJsonFactory().setCodec(smileMapper);
} else if (jsonMapper == null || smileMapper == null) {
throw new ISE(
"Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.",
jsonMapper,
smileMapper
);
}
if (lifecycle == null) {
@ -783,7 +756,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
configFactory = Config.createFactory(props);
}
return new IndexerCoordinatorNode(props, lifecycle, jsonMapper, configFactory);
return new IndexerCoordinatorNode(props, lifecycle, jsonMapper, smileMapper, configFactory);
}
}
}

View File

@ -177,7 +177,7 @@ public class IndexerCoordinatorResource
}
// Legacy endpoint
// TODO Remove
// TODO remove
@Deprecated
@GET
@Path("/status/{taskid}")
@ -239,7 +239,9 @@ public class IndexerCoordinatorResource
{
final Map<String, Object> retMap;
// TODO make sure this worker is supposed to be running this task (attempt id? token?)
// It would be great to verify that this worker is actually supposed to be running the task before
// actually doing the task. Some ideas for how that could be done would be using some sort of attempt_id
// or token that gets passed around.
try {
final T ret = taskActionClient.submit(holder.getAction());

View File

@ -29,9 +29,9 @@ import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.state.ConnectionState;
import com.netflix.curator.framework.state.ConnectionStateListener;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.zookeeper.CreateMode;
import org.joda.time.DateTime;

View File

@ -30,10 +30,10 @@ import com.metamx.druid.merger.coordinator.TaskRunner;
import com.metamx.druid.merger.coordinator.TaskRunnerFactory;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.emitter.EmittingLogger;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
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.commons.io.FileUtils;
import java.io.File;

View File

@ -24,6 +24,8 @@ import com.metamx.common.logger.Logger;
import com.metamx.druid.log.LogLevelAdjuster;
import java.io.File;
import java.util.Arrays;
import java.util.Iterator;
/**
*/
@ -35,16 +37,21 @@ public class ExecutorMain
{
LogLevelAdjuster.register();
if (args.length != 2) {
if (args.length != 3) {
log.info("Usage: ExecutorMain <task.json> <status.json>");
System.exit(2);
}
Iterator<String> arguments = Arrays.asList(args).iterator();
final String taskJsonFile = arguments.next();
final String statusJsonFile = arguments.next();
final ExecutorNode node = ExecutorNode.builder()
.build(
System.getProperty("druid.executor.nodeType", "indexer-executor"),
new ExecutorLifecycleFactory(
new File(args[0]),
new File(args[1]),
new File(taskJsonFile),
new File(statusJsonFile),
System.in
)
);

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
@ -34,13 +33,8 @@ import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.BaseServerNode;
import com.metamx.druid.client.ClientConfig;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.client.OnlyNewSegmentWatcherServerView;
import com.metamx.druid.http.QueryServlet;
import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.CuratorConfig;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServerInit;
@ -50,18 +44,13 @@ import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.ExecutorServiceTaskRunner;
import com.metamx.druid.merger.worker.config.WorkerConfig;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncerConfig;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters;
@ -74,9 +63,8 @@ import com.metamx.metrics.Monitor;
import com.metamx.metrics.MonitorScheduler;
import com.metamx.metrics.MonitorSchedulerConfig;
import com.metamx.metrics.SysMonitor;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.x.discovery.ServiceDiscovery;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.apache.curator.x.discovery.ServiceDiscovery;
import org.apache.curator.x.discovery.ServiceProvider;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.security.AWSCredentials;
@ -85,7 +73,6 @@ import org.mortbay.jetty.servlet.Context;
import org.mortbay.jetty.servlet.ServletHolder;
import org.skife.config.ConfigurationObjectFactory;
import java.io.IOException;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.Executors;
@ -115,15 +102,14 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
private WorkerConfig workerConfig = null;
private DataSegmentPusher segmentPusher = null;
private TaskToolboxFactory taskToolboxFactory = null;
private CuratorFramework curatorFramework = null;
private ServiceDiscovery serviceDiscovery = null;
private ServiceProvider coordinatorServiceProvider = null;
private MutableServerView newSegmentServerView = null;
private Server server = null;
private ExecutorServiceTaskRunner taskRunner = null;
private ExecutorLifecycle executorLifecycle = null;
public ExecutorNode(
String nodeType,
Properties props,
Lifecycle lifecycle,
ObjectMapper jsonMapper,
@ -132,7 +118,7 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
ExecutorLifecycleFactory executorLifecycleFactory
)
{
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
super(nodeType, log, props, lifecycle, jsonMapper, smileMapper, configFactory);
this.lifecycle = lifecycle;
this.props = props;
@ -170,12 +156,6 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
return this;
}
public ExecutorNode setCuratorFramework(CuratorFramework curatorFramework)
{
this.curatorFramework = curatorFramework;
return this;
}
public ExecutorNode setCoordinatorServiceProvider(ServiceProvider coordinatorServiceProvider)
{
this.coordinatorServiceProvider = coordinatorServiceProvider;
@ -188,12 +168,6 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
return this;
}
public ExecutorNode setNewSegmentServerView(MutableServerView newSegmentServerView)
{
this.newSegmentServerView = newSegmentServerView;
return this;
}
@Override
public void doInit() throws Exception
{
@ -202,10 +176,8 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
initializeS3Service();
initializeMonitors();
initializeMergerConfig();
initializeCuratorFramework();
initializeServiceDiscovery();
initializeCoordinatorServiceProvider();
initializeNewSegmentServerView();
initializeDataSegmentPusher();
initializeTaskToolbox();
initializeTaskRunner();
@ -216,10 +188,7 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
final MonitorScheduler monitorScheduler = new MonitorScheduler(
configFactory.build(MonitorSchedulerConfig.class),
globalScheduledExec,
emitter,
monitors
configFactory.build(MonitorSchedulerConfig.class), globalScheduledExec, emitter, monitors
);
lifecycle.addManagedInstance(monitorScheduler);
@ -370,11 +339,6 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
{
if (taskToolboxFactory == null) {
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
final SegmentAnnouncer segmentAnnouncer = new ZkSegmentAnnouncer(
configFactory.build(ZkSegmentAnnouncerConfig.class),
getPhoneBook()
);
lifecycle.addManagedInstance(segmentAnnouncer);
taskToolboxFactory = new TaskToolboxFactory(
taskConfig,
new RemoteTaskActionClientFactory(
@ -392,33 +356,20 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
s3Service,
segmentPusher,
dataSegmentKiller,
segmentAnnouncer,
newSegmentServerView,
getAnnouncer(),
getServerInventoryView(),
getConglomerate(),
getJsonMapper()
);
}
}
public void initializeCuratorFramework() throws IOException
{
if (curatorFramework == null) {
final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class);
curatorFramework = Initialization.makeCuratorFrameworkClient(
curatorConfig,
lifecycle
);
}
}
public void initializeServiceDiscovery() throws Exception
{
if (serviceDiscovery == null) {
final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class);
this.serviceDiscovery = Initialization.makeServiceDiscoveryClient(
curatorFramework,
config,
lifecycle
getCuratorFramework(), config, lifecycle
);
}
}
@ -434,21 +385,6 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
}
}
private void initializeNewSegmentServerView()
{
if (newSegmentServerView == null) {
final MutableServerView view = new OnlyNewSegmentWatcherServerView();
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
getConfigFactory().build(ClientConfig.class),
getPhoneBook(),
view
);
lifecycle.addManagedInstance(clientInventoryManager);
this.newSegmentServerView = view;
}
}
public void initializeTaskRunner()
{
if (taskRunner == null) {
@ -461,7 +397,7 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
.build()
)
)
);;
);
}
}
@ -497,7 +433,7 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
return this;
}
public ExecutorNode build(ExecutorLifecycleFactory executorLifecycleFactory)
public ExecutorNode build(String nodeType, ExecutorLifecycleFactory executorLifecycleFactory)
{
if (jsonMapper == null && smileMapper == null) {
jsonMapper = new DefaultObjectMapper();
@ -520,7 +456,7 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
configFactory = Config.createFactory(props);
}
return new ExecutorNode(props, lifecycle, jsonMapper, smileMapper, configFactory, executorLifecycleFactory);
return new ExecutorNode(nodeType, props, lifecycle, jsonMapper, smileMapper, configFactory, executorLifecycleFactory);
}
}
}

View File

@ -20,21 +20,21 @@
package com.metamx.druid.merger.worker.http;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.collect.Lists;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.servlet.GuiceFilter;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.config.Config;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.RegisteringNode;
import com.metamx.druid.QueryableNode;
import com.metamx.druid.http.GuiceServletConfig;
import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.CuratorConfig;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
@ -63,10 +63,10 @@ import com.metamx.metrics.Monitor;
import com.metamx.metrics.MonitorScheduler;
import com.metamx.metrics.MonitorSchedulerConfig;
import com.metamx.metrics.SysMonitor;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import com.netflix.curator.x.discovery.ServiceDiscovery;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.x.discovery.ServiceDiscovery;
import org.apache.curator.x.discovery.ServiceProvider;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.security.AWSCredentials;
@ -77,7 +77,6 @@ import org.mortbay.jetty.servlet.DefaultServlet;
import org.mortbay.jetty.servlet.ServletHolder;
import org.skife.config.ConfigurationObjectFactory;
import java.io.IOException;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutorService;
@ -86,7 +85,7 @@ import java.util.concurrent.ScheduledExecutorService;
/**
*/
public class WorkerNode extends RegisteringNode
public class WorkerNode extends QueryableNode<WorkerNode>
{
private static final EmittingLogger log = new EmittingLogger(WorkerNode.class);
@ -95,17 +94,11 @@ public class WorkerNode extends RegisteringNode
return new Builder();
}
private final Lifecycle lifecycle;
private final Properties props;
private final ObjectMapper jsonMapper;
private final ConfigurationObjectFactory configFactory;
private RestS3Service s3Service = null;
private List<Monitor> monitors = null;
private HttpClient httpClient = null;
private ServiceEmitter emitter = null;
private WorkerConfig workerConfig = null;
private CuratorFramework curatorFramework = null;
private ServiceDiscovery serviceDiscovery = null;
private ServiceProvider coordinatorServiceProvider = null;
private WorkerCuratorCoordinator workerCuratorCoordinator = null;
@ -120,15 +113,11 @@ public class WorkerNode extends RegisteringNode
Properties props,
Lifecycle lifecycle,
ObjectMapper jsonMapper,
ObjectMapper smileMapper,
ConfigurationObjectFactory configFactory
)
{
super(ImmutableList.of(jsonMapper));
this.lifecycle = lifecycle;
this.props = props;
this.jsonMapper = jsonMapper;
this.configFactory = configFactory;
super("indexer-worker", log, props, lifecycle, jsonMapper, smileMapper, configFactory);
}
public WorkerNode setHttpClient(HttpClient httpClient)
@ -143,9 +132,9 @@ public class WorkerNode extends RegisteringNode
return this;
}
public WorkerNode setCuratorFramework(CuratorFramework curatorFramework)
public WorkerNode setS3Service(RestS3Service s3Service)
{
this.curatorFramework = curatorFramework;
this.s3Service = s3Service;
return this;
}
@ -185,7 +174,6 @@ public class WorkerNode extends RegisteringNode
initializeEmitter();
initializeMonitors();
initializeMergerConfig();
initializeCuratorFramework();
initializeServiceDiscovery();
initializeCoordinatorServiceProvider();
initializeJacksonSubtypes();
@ -195,15 +183,15 @@ public class WorkerNode extends RegisteringNode
initializeWorkerTaskMonitor();
initializeServer();
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(getLifecycle());
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
final MonitorScheduler monitorScheduler = new MonitorScheduler(
configFactory.build(MonitorSchedulerConfig.class),
getConfigFactory().build(MonitorSchedulerConfig.class),
globalScheduledExec,
emitter,
monitors
);
lifecycle.addManagedInstance(monitorScheduler);
getLifecycle().addManagedInstance(monitorScheduler);
final Injector injector = Guice.createInjector(
new WorkerServletModule(
@ -227,26 +215,21 @@ public class WorkerNode extends RegisteringNode
doInit();
}
lifecycle.start();
getLifecycle().start();
}
@LifecycleStop
public synchronized void stop()
{
lifecycle.stop();
}
private ObjectMapper getJsonMapper()
{
return jsonMapper;
getLifecycle().stop();
}
private void initializeServer()
{
if (server == null) {
server = Initialization.makeJettyServer(configFactory.build(ServerConfig.class));
server = Initialization.makeJettyServer(getConfigFactory().build(ServerConfig.class));
lifecycle.addHandler(
getLifecycle().addHandler(
new Lifecycle.Handler()
{
@Override
@ -282,8 +265,8 @@ public class WorkerNode extends RegisteringNode
if (httpClient == null) {
httpClient = HttpClientInit.createClient(
HttpClientConfig.builder().withNumConnections(1)
.withReadTimeout(new Duration(PropUtils.getProperty(props, "druid.emitter.timeOut")))
.build(), lifecycle
.withReadTimeout(new Duration(PropUtils.getProperty(getProps(), "druid.emitter.timeOut")))
.build(), getLifecycle()
);
}
}
@ -292,9 +275,9 @@ public class WorkerNode extends RegisteringNode
{
if (emitter == null) {
emitter = new ServiceEmitter(
PropUtils.getProperty(props, "druid.service"),
PropUtils.getProperty(props, "druid.host"),
Emitters.create(props, httpClient, getJsonMapper(), lifecycle)
PropUtils.getProperty(getProps(), "druid.service"),
PropUtils.getProperty(getProps(), "druid.host"),
Emitters.create(getProps(), httpClient, getJsonMapper(), getLifecycle())
);
}
EmittingLogger.registerEmitter(emitter);
@ -305,8 +288,8 @@ public class WorkerNode extends RegisteringNode
if (s3Service == null) {
s3Service = new RestS3Service(
new AWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
PropUtils.getProperty(getProps(), "com.metamx.aws.accessKey"),
PropUtils.getProperty(getProps(), "com.metamx.aws.secretKey")
)
);
}
@ -324,29 +307,18 @@ public class WorkerNode extends RegisteringNode
private void initializeMergerConfig()
{
if (workerConfig == null) {
workerConfig = configFactory.build(WorkerConfig.class);
}
}
public void initializeCuratorFramework() throws IOException
{
if (curatorFramework == null) {
final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class);
curatorFramework = Initialization.makeCuratorFrameworkClient(
curatorConfig,
lifecycle
);
workerConfig = getConfigFactory().build(WorkerConfig.class);
}
}
public void initializeServiceDiscovery() throws Exception
{
if (serviceDiscovery == null) {
final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class);
final ServiceDiscoveryConfig config = getConfigFactory().build(ServiceDiscoveryConfig.class);
this.serviceDiscovery = Initialization.makeServiceDiscoveryClient(
curatorFramework,
getCuratorFramework(),
config,
lifecycle
getLifecycle()
);
}
}
@ -357,7 +329,7 @@ public class WorkerNode extends RegisteringNode
this.coordinatorServiceProvider = Initialization.makeServiceProvider(
workerConfig.getMasterService(),
serviceDiscovery,
lifecycle
getLifecycle()
);
}
}
@ -367,18 +339,18 @@ public class WorkerNode extends RegisteringNode
if (workerCuratorCoordinator == null) {
workerCuratorCoordinator = new WorkerCuratorCoordinator(
getJsonMapper(),
configFactory.build(IndexerZkConfig.class),
curatorFramework,
getConfigFactory().build(IndexerZkConfig.class),
getCuratorFramework(),
new Worker(workerConfig)
);
lifecycle.addManagedInstance(workerCuratorCoordinator);
getLifecycle().addManagedInstance(workerCuratorCoordinator);
}
}
private void initializePersistentTaskLogs() throws S3ServiceException
{
if (persistentTaskLogs == null) {
final TaskLogConfig taskLogConfig = configFactory.build(TaskLogConfig.class);
final TaskLogConfig taskLogConfig = getConfigFactory().build(TaskLogConfig.class);
if (taskLogConfig.getLogStorageBucket() != null) {
initializeS3Service();
persistentTaskLogs = new S3TaskLogs(
@ -396,8 +368,8 @@ public class WorkerNode extends RegisteringNode
{
if (forkingTaskRunner == null) {
forkingTaskRunner = new ForkingTaskRunner(
configFactory.build(ForkingTaskRunnerConfig.class),
props,
getConfigFactory().build(ForkingTaskRunnerConfig.class),
getProps(),
persistentTaskLogs,
Executors.newFixedThreadPool(workerConfig.getCapacity()),
getJsonMapper()
@ -409,6 +381,8 @@ public class WorkerNode extends RegisteringNode
{
if (workerTaskMonitor == null) {
final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getNumThreads());
final CuratorFramework curatorFramework = getCuratorFramework();
final PathChildrenCache pathChildrenCache = new PathChildrenCache(
curatorFramework,
workerCuratorCoordinator.getTaskPathForWorker(),
@ -422,13 +396,14 @@ public class WorkerNode extends RegisteringNode
forkingTaskRunner,
workerExec
);
lifecycle.addManagedInstance(workerTaskMonitor);
getLifecycle().addManagedInstance(workerTaskMonitor);
}
}
public static class Builder
{
private ObjectMapper jsonMapper = null;
private ObjectMapper smileMapper = null;
private Lifecycle lifecycle = null;
private Properties props = null;
private ConfigurationObjectFactory configFactory = null;
@ -459,8 +434,16 @@ public class WorkerNode extends RegisteringNode
public WorkerNode build()
{
if (jsonMapper == null) {
if (jsonMapper == null && smileMapper == null) {
jsonMapper = new DefaultObjectMapper();
smileMapper = new DefaultObjectMapper(new SmileFactory());
smileMapper.getJsonFactory().setCodec(smileMapper);
} else if (jsonMapper == null || smileMapper == null) {
throw new ISE(
"Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.",
jsonMapper,
smileMapper
);
}
if (lifecycle == null) {
@ -475,7 +458,7 @@ public class WorkerNode extends RegisteringNode
configFactory = Config.createFactory(props);
}
return new WorkerNode(props, lifecycle, jsonMapper, configFactory);
return new WorkerNode(props, lifecycle, jsonMapper, smileMapper, configFactory);
}
}
}

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.worker.http;
import com.fasterxml.jackson.databind.ObjectMapper;

View File

@ -24,11 +24,11 @@ import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
import com.metamx.druid.merger.worker.WorkerTaskMonitor;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.CuratorFrameworkFactory;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import com.netflix.curator.retry.ExponentialBackoffRetry;
import com.netflix.curator.test.TestingCluster;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.retry.ExponentialBackoffRetry;
import org.apache.curator.test.TestingCluster;
import org.apache.commons.lang.mutable.MutableBoolean;
import org.apache.zookeeper.CreateMode;
import org.easymock.EasyMock;

39
pom.xml
View File

@ -38,7 +38,8 @@
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.21.0</metamx.java-util.version>
<metamx.java-util.version>0.22.0</metamx.java-util.version>
<netflix.curator.version>2.0.1-21-22</netflix.curator.version>
</properties>
<modules>
@ -118,29 +119,24 @@
<version>0.9</version>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-client</artifactId>
<version>1.3.2</version>
<version>${netflix.curator.version}</version>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-framework</artifactId>
<version>1.3.2</version>
<version>${netflix.curator.version}</version>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-recipes</artifactId>
<version>1.3.2</version>
<version>${netflix.curator.version}</version>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-x-discovery</artifactId>
<version>1.3.2</version>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<artifactId>curator-test</artifactId>
<version>1.2.2</version>
<version>${netflix.curator.version}</version>
</dependency>
<dependency>
<groupId>it.uniroma3.mat</groupId>
@ -283,11 +279,6 @@
<artifactId>slf4j-log4j12</artifactId>
<version>1.6.2</version>
</dependency>
<dependency>
<groupId>com.github.sgroschupf</groupId>
<artifactId>zkclient</artifactId>
<version>0.1</version>
</dependency>
<dependency>
<groupId>com.google.code.simple-spring-memcached</groupId>
<artifactId>spymemcached</artifactId>
@ -308,6 +299,9 @@
<artifactId>commons-cli</artifactId>
<version>1.2</version>
</dependency>
<!-- Test Scope -->
<dependency>
<groupId>com.metamx</groupId>
@ -316,7 +310,6 @@
<scope>test</scope>
<version>${metamx.java-util.version}</version>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
@ -335,6 +328,12 @@
<version>0.5-rc1</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.curator</groupId>
<artifactId>curator-test</artifactId>
<version>${netflix.curator.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</dependencyManagement>

View File

@ -120,16 +120,6 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.github.sgroschupf</groupId>
<artifactId>zkclient</artifactId>
<exclusions>
<exclusion>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
</exclusion>
</exclusions>
</dependency>
<!-- Dependencies required for jets3t b/c emr pom doesn't include them -->
<dependency>

View File

@ -17,24 +17,10 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.client;
import org.skife.config.Config;
package com.metamx.druid.realtime;
/**
*/
public abstract class ClientConfig extends InventoryManagerConfig
public abstract class RealtimeCuratorDataSegmentAnnouncerConfig
{
public ClientConfig()
{
super(null, null);
}
@Override
@Config("druid.zk.paths.announcementsPath")
public abstract String getInventoryIdPath();
@Override
@Config("druid.zk.paths.servedSegmentsPath")
public abstract String getInventoryPath();
}

View File

@ -35,11 +35,6 @@ import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.BaseServerNode;
import com.metamx.druid.client.ClientConfig;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.client.OnlyNewSegmentWatcherServerView;
import com.metamx.druid.client.ServerView;
import com.metamx.druid.db.DbConnector;
import com.metamx.druid.db.DbConnectorConfig;
import com.metamx.druid.http.QueryServlet;
@ -52,7 +47,6 @@ import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.metrics.Monitor;
import org.mortbay.jetty.servlet.Context;
import org.mortbay.jetty.servlet.ServletHolder;
import org.skife.config.ConfigurationObjectFactory;
@ -76,11 +70,9 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
private final Map<String, Object> injectablesMap = Maps.newLinkedHashMap();
private SegmentAnnouncer segmentAnnouncer = null;
private SegmentPublisher segmentPublisher = null;
private DataSegmentPusher dataSegmentPusher = null;
private List<FireDepartment> fireDepartments = null;
private ServerView view = null;
private boolean initialized = false;
@ -92,41 +84,24 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
ConfigurationObjectFactory configFactory
)
{
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
}
public RealtimeNode setView(ServerView view)
{
Preconditions.checkState(this.view == null, "Cannot set view once it has already been set.");
this.view = view;
return this;
}
public RealtimeNode setSegmentAnnouncer(SegmentAnnouncer segmentAnnouncer)
{
Preconditions.checkState(this.segmentAnnouncer == null, "Cannot set segmentAnnouncer once it has already been set.");
this.segmentAnnouncer = segmentAnnouncer;
return this;
super("realtime", log, props, lifecycle, jsonMapper, smileMapper, configFactory);
}
public RealtimeNode setSegmentPublisher(SegmentPublisher segmentPublisher)
{
Preconditions.checkState(this.segmentPublisher == null, "Cannot set segmentPublisher once it has already been set.");
this.segmentPublisher = segmentPublisher;
checkFieldNotSetAndSet("segmentPublisher", segmentPublisher);
return this;
}
public RealtimeNode setDataSegmentPusher(DataSegmentPusher dataSegmentPusher)
{
Preconditions.checkState(this.dataSegmentPusher == null, "Cannot set segmentPusher once it has already been set.");
this.dataSegmentPusher = dataSegmentPusher;
checkFieldNotSetAndSet("dataSegmentPusher", dataSegmentPusher);
return this;
}
public RealtimeNode setFireDepartments(List<FireDepartment> fireDepartments)
{
Preconditions.checkState(this.fireDepartments == null, "Cannot set fireDepartments once it has already been set.");
this.fireDepartments = fireDepartments;
checkFieldNotSetAndSet("fireDepartments", fireDepartments);
return this;
}
@ -137,12 +112,6 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
return this;
}
public SegmentAnnouncer getSegmentAnnouncer()
{
initializeSegmentAnnouncer();
return segmentAnnouncer;
}
public SegmentPublisher getSegmentPublisher()
{
initializeSegmentPublisher();
@ -161,30 +130,19 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
return fireDepartments;
}
public ServerView getView()
{
initializeView();
return view;
}
protected void doInit() throws Exception
{
initializeView();
initializeSegmentAnnouncer();
initializeSegmentPublisher();
initializeSegmentPusher();
initializeJacksonInjectables();
initializeFireDepartments();
final Lifecycle lifecycle = getLifecycle();
final ServiceEmitter emitter = getEmitter();
final QueryRunnerFactoryConglomerate conglomerate = getConglomerate();
final List<Monitor> monitors = getMonitors();
final List<FireDepartment> departments = getFireDepartments();
monitors.add(new RealtimeMetricsMonitor(fireDepartments));
monitors.add(new RealtimeMetricsMonitor(departments));
final RealtimeManager realtimeManager = new RealtimeManager(fireDepartments, conglomerate);
final RealtimeManager realtimeManager = new RealtimeManager(departments, conglomerate);
lifecycle.addManagedInstance(realtimeManager);
startMonitoring(monitors);
@ -226,10 +184,10 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
}
injectables.put("queryRunnerFactoryConglomerate", getConglomerate());
injectables.put("segmentPusher", dataSegmentPusher);
injectables.put("segmentAnnouncer", segmentAnnouncer);
injectables.put("segmentPublisher", segmentPublisher);
injectables.put("serverView", view);
injectables.put("segmentPusher", getDataSegmentPusher());
injectables.put("segmentAnnouncer", getAnnouncer());
injectables.put("segmentPublisher", getSegmentPublisher());
injectables.put("serverView", getServerInventoryView());
injectables.put("serviceEmitter", getEmitter());
getJsonMapper().setInjectableValues(
@ -250,9 +208,11 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
{
if (fireDepartments == null) {
try {
fireDepartments = getJsonMapper().readValue(
new File(PropUtils.getProperty(getProps(), "druid.realtime.specFile")),
new TypeReference<List<FireDepartment>>(){}
setFireDepartments(
getJsonMapper().<List<FireDepartment>>readValue(
new File(PropUtils.getProperty(getProps(), "druid.realtime.specFile")),
new TypeReference<List<FireDepartment>>(){}
)
);
}
catch (IOException e) {
@ -268,15 +228,6 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
}
}
protected void initializeSegmentAnnouncer()
{
if (segmentAnnouncer == null) {
final ZkSegmentAnnouncerConfig zkSegmentAnnouncerConfig = getConfigFactory().build(ZkSegmentAnnouncerConfig.class);
segmentAnnouncer = new ZkSegmentAnnouncer(zkSegmentAnnouncerConfig, getPhoneBook());
getLifecycle().addManagedInstance(segmentAnnouncer);
}
}
protected void initializeSegmentPublisher()
{
if (segmentPublisher == null) {
@ -290,21 +241,6 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
}
}
private void initializeView()
{
if (view == null) {
final MutableServerView view = new OnlyNewSegmentWatcherServerView();
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
getConfigFactory().build(ClientConfig.class),
getPhoneBook(),
view
);
getLifecycle().addManagedInstance(clientInventoryManager);
this.view = view;
}
}
public static class Builder
{
private ObjectMapper jsonMapper = null;

View File

@ -1,104 +0,0 @@
package com.metamx.druid.realtime;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.phonebook.PhoneBook;
import org.joda.time.DateTime;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
public class ZkSegmentAnnouncer implements SegmentAnnouncer
{
private static final Logger log = new Logger(ZkSegmentAnnouncer.class);
private final Object lock = new Object();
private final ZkSegmentAnnouncerConfig config;
private final PhoneBook yp;
private final String servedSegmentsLocation;
private volatile boolean started = false;
public ZkSegmentAnnouncer(
ZkSegmentAnnouncerConfig config,
PhoneBook yp
)
{
this.config = config;
this.yp = yp;
this.servedSegmentsLocation = yp.combineParts(
Arrays.asList(
config.getServedSegmentsLocation(), config.getServerName()
)
);
}
public Map<String, String> getStringProps()
{
return ImmutableMap.of(
"name", config.getServerName(),
"host", config.getHost(),
"maxSize", String.valueOf(config.getMaxSize()),
"type", "realtime"
);
}
@LifecycleStart
public void start()
{
synchronized (lock) {
if (started) {
return;
}
log.info("Starting zkCoordinator for server[%s] with config[%s]", config.getServerName(), config);
if (yp.lookup(servedSegmentsLocation, Object.class) == null) {
yp.post(
config.getServedSegmentsLocation(),
config.getServerName(),
ImmutableMap.of("created", new DateTime().toString())
);
}
yp.announce(
config.getAnnounceLocation(),
config.getServerName(),
getStringProps()
);
started = true;
}
}
@LifecycleStop
public void stop()
{
synchronized (lock) {
if (!started) {
return;
}
log.info("Stopping ZkSegmentAnnouncer with config[%s]", config);
yp.unannounce(config.getAnnounceLocation(), config.getServerName());
started = false;
}
}
public void announceSegment(DataSegment segment) throws IOException
{
log.info("Announcing realtime segment %s", segment.getIdentifier());
yp.announce(servedSegmentsLocation, segment.getIdentifier(), segment);
}
public void unannounceSegment(DataSegment segment) throws IOException
{
log.info("Unannouncing realtime segment %s", segment.getIdentifier());
yp.unannounce(servedSegmentsLocation, segment.getIdentifier());
}
}

View File

@ -1,23 +0,0 @@
package com.metamx.druid.realtime;
import org.skife.config.Config;
import org.skife.config.Default;
public abstract class ZkSegmentAnnouncerConfig
{
@Config("druid.host")
public abstract String getServerName();
@Config("druid.host")
public abstract String getHost();
@Config("druid.server.maxSize")
@Default("0")
public abstract long getMaxSize();
@Config("druid.zk.paths.announcementsPath")
public abstract String getAnnounceLocation();
@Config("druid.zk.paths.servedSegmentsPath")
public abstract String getServedSegmentsLocation();
}

View File

@ -40,6 +40,7 @@ import com.metamx.druid.Query;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.DruidServer;
import com.metamx.druid.client.ServerView;
import com.metamx.druid.coordination.DataSegmentAnnouncer;
import com.metamx.druid.guava.ThreadRenamingCallable;
import com.metamx.druid.guava.ThreadRenamingRunnable;
import com.metamx.druid.index.QueryableIndex;
@ -57,7 +58,6 @@ import com.metamx.druid.query.QueryToolChest;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.FireHydrant;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.SegmentPublisher;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
@ -97,7 +97,7 @@ public class RealtimePlumberSchool implements PlumberSchool
private volatile RejectionPolicyFactory rejectionPolicyFactory = null;
private volatile QueryRunnerFactoryConglomerate conglomerate = null;
private volatile DataSegmentPusher dataSegmentPusher = null;
private volatile SegmentAnnouncer segmentAnnouncer = null;
private volatile DataSegmentAnnouncer segmentAnnouncer = null;
private volatile SegmentPublisher segmentPublisher = null;
private volatile ServerView serverView = null;
@ -144,7 +144,7 @@ public class RealtimePlumberSchool implements PlumberSchool
}
@JacksonInject("segmentAnnouncer")
public void setSegmentAnnouncer(SegmentAnnouncer segmentAnnouncer)
public void setSegmentAnnouncer(DataSegmentAnnouncer segmentAnnouncer)
{
this.segmentAnnouncer = segmentAnnouncer;
}

View File

@ -81,11 +81,11 @@
<artifactId>config-magic</artifactId>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-framework</artifactId>
</dependency>
<dependency>
<groupId>com.netflix.curator</groupId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-x-discovery</artifactId>
</dependency>
<dependency>
@ -168,10 +168,6 @@
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</dependency>
<dependency>
<groupId>com.github.sgroschupf</groupId>
<artifactId>zkclient</artifactId>
</dependency>
<!-- Dependencies required for jets3t b/c emr pom doesn't include them -->
<dependency>
@ -214,6 +210,20 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.metamx.druid</groupId>
<artifactId>druid-client</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.curator</groupId>
<artifactId>curator-test</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

View File

@ -47,6 +47,7 @@ public abstract class BaseServerNode<T extends QueryableNode> extends QueryableN
private StupidPool<ByteBuffer> computeScratchPool = null;
public BaseServerNode(
String nodeType,
Logger log,
Properties props,
Lifecycle lifecycle,
@ -55,7 +56,7 @@ public abstract class BaseServerNode<T extends QueryableNode> extends QueryableN
ConfigurationObjectFactory configFactory
)
{
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
super(nodeType, log, props, lifecycle, jsonMapper, smileMapper, configFactory);
}
public QueryRunnerFactoryConglomerate getConglomerate()

View File

@ -28,10 +28,10 @@ import com.metamx.druid.client.DataSegment;
@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="action")
@JsonSubTypes(value={
@JsonSubTypes.Type(name="load", value=SegmentChangeRequestLoad.class),
@JsonSubTypes.Type(name="drop", value=SegmentChangeRequestDrop.class)
@JsonSubTypes.Type(name="drop", value=SegmentChangeRequestDrop.class),
@JsonSubTypes.Type(name="noop", value=SegmentChangeRequestNoop.class)
})
public interface DataSegmentChangeRequest
{
public void go(DataSegmentChangeHandler handler);
public DataSegment getSegment();
}

View File

@ -1,55 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.coordination;
import com.metamx.druid.master.DruidMaster;
import com.metamx.phonebook.PhoneBook;
import java.util.Arrays;
import java.util.Map;
/**
*/
public class DruidClusterInfo
{
private final DruidClusterInfoConfig config;
private final PhoneBook yp;
public DruidClusterInfo(
DruidClusterInfoConfig config,
PhoneBook zkPhoneBook
)
{
this.config = config;
this.yp = zkPhoneBook;
}
public Map<String, String> lookupCurrentLeader()
{
return (Map<String, String>) yp.lookup(
yp.combineParts(Arrays.asList(config.getMasterPath(), DruidMaster.MASTER_OWNER_NODE)), Map.class
);
}
public String getMasterHost()
{
return lookupCurrentLeader().get("host");
}
}

View File

@ -40,7 +40,6 @@ public class SegmentChangeRequestDrop implements DataSegmentChangeRequest
@JsonProperty
@JsonUnwrapped
@Override
public DataSegment getSegment()
{
return segment;

View File

@ -46,7 +46,6 @@ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest
@JsonProperty
@JsonUnwrapped
@Override
public DataSegment getSegment()
{
return segment;

View File

@ -19,12 +19,13 @@
package com.metamx.druid.coordination;
import org.skife.config.Config;
/**
*/
public abstract class DruidClusterInfoConfig
public class SegmentChangeRequestNoop implements DataSegmentChangeRequest
{
@Config("druid.zk.paths.masterPath")
public abstract String getMasterPath();
@Override
public void go(DataSegmentChangeHandler handler)
{
}
}

View File

@ -21,164 +21,141 @@ package com.metamx.druid.coordination;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.metamx.common.Pair;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.DruidServer;
import com.metamx.druid.initialization.ZkPathsConfig;
import com.metamx.druid.loading.SegmentLoadingException;
import com.metamx.emitter.service.AlertEvent;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.phonebook.PhoneBook;
import com.metamx.phonebook.PhoneBookPeon;
import org.joda.time.DateTime;
import com.metamx.emitter.EmittingLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.ChildData;
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.curator.utils.ZKPaths;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
/**
*/
public class ZkCoordinator implements DataSegmentChangeHandler
{
private static final Logger log = new Logger(ZkCoordinator.class);
private static final EmittingLogger log = new EmittingLogger(ZkCoordinator.class);
private final Object lock = new Object();
private final ObjectMapper jsonMapper;
private final ZkCoordinatorConfig config;
private final DruidServer me;
private final PhoneBook yp;
private final DruidServerMetadata me;
private final DataSegmentAnnouncer announcer;
private final CuratorFramework curator;
private final ServerManager serverManager;
private final ServiceEmitter emitter;
private final List<Pair<String, PhoneBookPeon<?>>> peons;
private final String loadQueueLocation;
private final String servedSegmentsLocation;
private volatile PathChildrenCache loadQueueCache;
private volatile boolean started;
public ZkCoordinator(
ObjectMapper jsonMapper,
ZkCoordinatorConfig config,
DruidServer me,
PhoneBook yp,
ServerManager serverManager,
ServiceEmitter emitter
ZkPathsConfig zkPaths,
DruidServerMetadata me,
DataSegmentAnnouncer announcer,
CuratorFramework curator,
ServerManager serverManager
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.me = me;
this.yp = yp;
this.announcer = announcer;
this.curator = curator;
this.serverManager = serverManager;
this.emitter = emitter;
this.peons = new ArrayList<Pair<String, PhoneBookPeon<?>>>();
this.loadQueueLocation = yp.combineParts(Arrays.asList(config.getLoadQueueLocation(), me.getName()));
this.servedSegmentsLocation = yp.combineParts(
Arrays.asList(
config.getServedSegmentsLocation(), me.getName()
)
);
this.config.getSegmentInfoCacheDirectory().mkdirs();
this.loadQueueLocation = ZKPaths.makePath(zkPaths.getLoadQueuePath(), me.getName());
this.servedSegmentsLocation = ZKPaths.makePath(zkPaths.getServedSegmentsPath(), me.getName());
}
@LifecycleStart
public void start() throws IOException
{
log.info("Starting zkCoordinator for server[%s] with config[%s]", me, config);
log.info("Starting zkCoordinator for server[%s]", me);
synchronized (lock) {
if (started) {
return;
}
if (yp.lookup(loadQueueLocation, Object.class) == null) {
yp.post(
config.getLoadQueueLocation(),
me.getName(),
ImmutableMap.of("created", new DateTime().toString())
);
}
if (yp.lookup(servedSegmentsLocation, Object.class) == null) {
yp.post(
config.getServedSegmentsLocation(),
me.getName(),
ImmutableMap.of("created", new DateTime().toString())
);
}
loadCache();
yp.announce(
config.getAnnounceLocation(),
me.getName(),
me.getStringProps()
loadQueueCache = new PathChildrenCache(
curator,
loadQueueLocation,
true,
true,
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ZkCoordinator-%s").build()
);
peons.add(
new Pair<String, PhoneBookPeon<?>>(
loadQueueLocation,
new PhoneBookPeon<DataSegmentChangeRequest>()
try {
config.getSegmentInfoCacheDirectory().mkdirs();
curator.newNamespaceAwareEnsurePath(loadQueueLocation).ensure(curator.getZookeeperClient());
curator.newNamespaceAwareEnsurePath(servedSegmentsLocation).ensure(curator.getZookeeperClient());
loadCache();
loadQueueCache.getListenable().addListener(
new PathChildrenCacheListener()
{
@Override
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
{
@Override
public Class<DataSegmentChangeRequest> getObjectClazz()
{
return DataSegmentChangeRequest.class;
}
@Override
public void newEntry(String nodeName, DataSegmentChangeRequest segment)
{
log.info("New node[%s] with segmentClass[%s]", nodeName, segment.getClass());
try {
segment.go(ZkCoordinator.this);
yp.unpost(loadQueueLocation, nodeName);
log.info("Completed processing for node[%s]", nodeName);
}
catch (Throwable t) {
yp.unpost(loadQueueLocation, nodeName);
log.error(
t, "Uncaught throwable made it through loading. Node[%s/%s]", loadQueueLocation, nodeName
);
Map<String, Object> exceptionMap = Maps.newHashMap();
exceptionMap.put("node", loadQueueLocation);
exceptionMap.put("nodeName", nodeName);
exceptionMap.put("nodeProperties", segment.toString());
exceptionMap.put("exception", t.getMessage());
emitter.emit(
new AlertEvent.Builder().build(
"Uncaught exception related to segment load/unload",
exceptionMap
)
final ChildData child = event.getData();
switch (event.getType()) {
case CHILD_ADDED:
final String path = child.getPath();
final DataSegmentChangeRequest segment = jsonMapper.readValue(
child.getData(), DataSegmentChangeRequest.class
);
throw Throwables.propagate(t);
}
}
log.info("New node[%s] with segmentClass[%s]", path, segment.getClass());
@Override
public void entryRemoved(String name)
{
log.info("%s was removed", name);
try {
segment.go(ZkCoordinator.this);
curator.delete().guaranteed().forPath(path);
log.info("Completed processing for node[%s]", path);
}
catch (Exception e) {
try {
curator.delete().guaranteed().forPath(path);
}
catch (Exception e1) {
log.info(e1, "Failed to delete node[%s], but ignoring exception.", path);
}
log.makeAlert(e, "Segment load/unload: uncaught exception.")
.addData("node", path)
.addData("nodeProperties", segment)
.emit();
}
break;
case CHILD_REMOVED:
log.info("%s was removed", event.getData().getPath());
break;
default:
log.info("Ignoring event[%s]", event);
}
}
)
);
for (Pair<String, PhoneBookPeon<?>> peon : peons) {
yp.registerListener(peon.lhs, peon.rhs);
}
);
loadQueueCache.start();
}
catch (Exception e) {
Throwables.propagateIfPossible(e, IOException.class);
throw Throwables.propagate(e);
}
started = true;
@ -194,14 +171,17 @@ public class ZkCoordinator implements DataSegmentChangeHandler
return;
}
for (Pair<String, PhoneBookPeon<?>> peon : peons) {
yp.unregisterListener(peon.lhs, peon.rhs);
try {
loadQueueCache.close();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
finally {
loadQueueCache = null;
started = false;
}
peons.clear();
yp.unannounce(config.getAnnounceLocation(), me.getName());
started = false;
}
}
@ -228,16 +208,9 @@ public class ZkCoordinator implements DataSegmentChangeHandler
}
}
catch (Exception e) {
log.error(e, "Exception occurred reading file [%s]", file);
emitter.emit(
new AlertEvent.Builder().build(
"Failed to read segment info file",
ImmutableMap.<String, Object>builder()
.put("file", file)
.put("exception", e.toString())
.build()
)
);
log.makeAlert(e, "Failed to load segment from segmentInfo file")
.addData("file", file)
.emit();
}
}
}
@ -255,23 +228,22 @@ public class ZkCoordinator implements DataSegmentChangeHandler
catch (IOException e) {
removeSegment(segment);
throw new SegmentLoadingException(
"Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile
e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile
);
}
yp.announce(servedSegmentsLocation, segment.getIdentifier(), segment);
try {
announcer.announceSegment(segment);
}
catch (IOException e) {
removeSegment(segment);
throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getIdentifier());
}
}
catch (SegmentLoadingException e) {
log.error(e, "Failed to load segment[%s]", segment);
emitter.emit(
new AlertEvent.Builder().build(
"Failed to load segment",
ImmutableMap.<String, Object>builder()
.put("segment", segment.toString())
.put("exception", e.toString())
.build()
)
);
log.makeAlert(e, "Failed to load segment for dataSource")
.addData("segment", segment)
.emit();
}
}
@ -286,19 +258,12 @@ public class ZkCoordinator implements DataSegmentChangeHandler
log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile);
}
yp.unannounce(servedSegmentsLocation, segment.getIdentifier());
announcer.unannounceSegment(segment);
}
catch (Exception e) {
log.error(e, "Exception thrown when dropping segment[%s]", segment);
emitter.emit(
new AlertEvent.Builder().build(
"Failed to remove segment",
ImmutableMap.<String, Object>builder()
.put("segment", segment.toString())
.put("exception", e.toString())
.build()
)
);
log.makeAlert("Failed to remove segment")
.addData("segment", segment)
.emit();
}
}
}

View File

@ -27,15 +27,6 @@ import java.io.File;
*/
public abstract class ZkCoordinatorConfig
{
@Config("druid.zk.paths.announcementsPath")
public abstract String getAnnounceLocation();
@Config("druid.zk.paths.servedSegmentsPath")
public abstract String getServedSegmentsLocation();
@Config("druid.zk.paths.loadQueuePath")
public abstract String getLoadQueueLocation();
@Config("druid.paths.segmentInfoCache")
public abstract File getSegmentInfoCacheDirectory();
}

View File

@ -121,9 +121,8 @@ public class DatabaseSegmentManager
return;
}
dataSources.set(new ConcurrentHashMap<String, DruidDataSource>());
started = false;
dataSources.set(new ConcurrentHashMap<String, DruidDataSource>());
}
}
@ -386,8 +385,11 @@ public class DatabaseSegmentManager
public void poll()
{
try {
ConcurrentHashMap<String, DruidDataSource> newDataSources
= new ConcurrentHashMap<String, DruidDataSource>();
if (!started) {
return;
}
ConcurrentHashMap<String, DruidDataSource> newDataSources = new ConcurrentHashMap<String, DruidDataSource>();
List<Map<String, Object>> segmentRows = dbi.withHandle(
new HandleCallback<List<Map<String, Object>>>()
@ -439,10 +441,14 @@ public class DatabaseSegmentManager
}
}
dataSources.set(newDataSources);
synchronized (lock) {
if (started) {
dataSources.set(newDataSources);
}
}
}
catch (Exception e) {
log.error(e, e.toString());
log.error(e, "Problem polling DB.");
}
}
}

View File

@ -31,8 +31,6 @@ import com.metamx.common.config.Config;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.BaseServerNode;
import com.metamx.druid.client.DruidServer;
import com.metamx.druid.client.DruidServerConfig;
import com.metamx.druid.coordination.ServerManager;
import com.metamx.druid.coordination.ZkCoordinator;
import com.metamx.druid.coordination.ZkCoordinatorConfig;
@ -70,7 +68,6 @@ public class ComputeNode extends BaseServerNode<ComputeNode>
return new Builder();
}
private DruidServer druidServer;
private SegmentLoader segmentLoader;
public ComputeNode(
@ -81,7 +78,7 @@ public class ComputeNode extends BaseServerNode<ComputeNode>
ConfigurationObjectFactory configFactory
)
{
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
super("historical", log, props, lifecycle, jsonMapper, smileMapper, configFactory);
}
public ComputeNode setSegmentLoader(SegmentLoader segmentLoader)
@ -91,30 +88,14 @@ public class ComputeNode extends BaseServerNode<ComputeNode>
return this;
}
public ComputeNode setDruidServer(DruidServer druidServer)
{
Preconditions.checkState(this.druidServer == null, "Cannot set druidServer once it has already been set.");
this.druidServer = druidServer;
return this;
}
public DruidServer getDruidServer()
{
initializeDruidServer();
return druidServer;
}
public SegmentLoader getSegmentLoader()
{
initializeAdapterLoader();
initializeSegmentLoader();
return segmentLoader;
}
protected void doInit() throws Exception
{
initializeDruidServer();
initializeAdapterLoader();
final Lifecycle lifecycle = getLifecycle();
final ServiceEmitter emitter = getEmitter();
final List<Monitor> monitors = getMonitors();
@ -131,19 +112,20 @@ public class ComputeNode extends BaseServerNode<ComputeNode>
new ServiceMetricEvent.Builder()
);
final ServerManager serverManager = new ServerManager(segmentLoader, conglomerate, emitter, executorService);
final ServerManager serverManager = new ServerManager(getSegmentLoader(), conglomerate, emitter, executorService);
final ZkCoordinator coordinator = new ZkCoordinator(
getJsonMapper(),
getConfigFactory().build(ZkCoordinatorConfig.class),
druidServer,
getPhoneBook(),
serverManager,
emitter
getZkPaths(),
getDruidServerMetadata(),
getAnnouncer(),
getCuratorFramework(),
serverManager
);
lifecycle.addManagedInstance(coordinator);
monitors.add(new ServerMonitor(getDruidServer(), serverManager));
monitors.add(new ServerMonitor(getDruidServerMetadata(), serverManager));
startMonitoring(monitors);
final Context root = new Context(getServer(), "/", Context.SESSIONS);
@ -156,7 +138,7 @@ public class ComputeNode extends BaseServerNode<ComputeNode>
);
}
private void initializeAdapterLoader()
private void initializeSegmentLoader()
{
if (segmentLoader == null) {
final Properties props = getProps();
@ -178,13 +160,6 @@ public class ComputeNode extends BaseServerNode<ComputeNode>
}
}
private void initializeDruidServer()
{
if (druidServer == null) {
setDruidServer(new DruidServer(getConfigFactory().build(DruidServerConfig.class), "historical"));
}
}
public static class Builder
{
private ObjectMapper jsonMapper = null;

View File

@ -29,12 +29,11 @@ import com.google.common.collect.Sets;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.DruidDataSource;
import com.metamx.druid.client.DruidServer;
import com.metamx.druid.client.ServerInventoryManager;
import com.metamx.druid.coordination.DruidClusterInfo;
import com.metamx.druid.client.ServerInventoryView;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.db.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseSegmentManager;
import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.master.rules.Rule;
import org.joda.time.Interval;
@ -62,27 +61,24 @@ import java.util.TreeSet;
public class InfoResource
{
private final DruidMaster master;
private final ServerInventoryManager serverInventoryManager;
private final ServerInventoryView serverInventoryView;
private final DatabaseSegmentManager databaseSegmentManager;
private final DatabaseRuleManager databaseRuleManager;
private final DruidClusterInfo druidClusterInfo;
private final IndexingServiceClient indexingServiceClient;
@Inject
public InfoResource(
DruidMaster master,
ServerInventoryManager serverInventoryManager,
ServerInventoryView serverInventoryView,
DatabaseSegmentManager databaseSegmentManager,
DatabaseRuleManager databaseRuleManager,
DruidClusterInfo druidClusterInfo,
IndexingServiceClient indexingServiceClient
)
{
this.master = master;
this.serverInventoryManager = serverInventoryManager;
this.serverInventoryView = serverInventoryView;
this.databaseSegmentManager = databaseSegmentManager;
this.databaseRuleManager = databaseRuleManager;
this.druidClusterInfo = druidClusterInfo;
this.indexingServiceClient = indexingServiceClient;
}
@ -92,7 +88,7 @@ public class InfoResource
public Response getMaster()
{
return Response.status(Response.Status.OK)
.entity(druidClusterInfo.lookupCurrentLeader())
.entity(master.getCurrentMaster())
.build();
}
@ -102,7 +98,7 @@ public class InfoResource
public Response getClusterInfo()
{
return Response.status(Response.Status.OK)
.entity(serverInventoryManager.getInventory())
.entity(serverInventoryView.getInventory())
.build();
}
@ -115,12 +111,12 @@ public class InfoResource
{
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
if (full != null) {
return builder.entity(serverInventoryManager.getInventory()).build();
return builder.entity(serverInventoryView.getInventory()).build();
}
return builder.entity(
Iterables.transform(
serverInventoryManager.getInventory(),
serverInventoryView.getInventory(),
new Function<DruidServer, String>()
{
@Override
@ -141,7 +137,7 @@ public class InfoResource
)
{
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
DruidServer server = serverInventoryManager.getInventoryValue(serverName);
DruidServer server = serverInventoryView.getInventoryValue(serverName);
if (server == null) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -160,7 +156,7 @@ public class InfoResource
)
{
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
DruidServer server = serverInventoryManager.getInventoryValue(serverName);
DruidServer server = serverInventoryView.getInventoryValue(serverName);
if (server == null) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -192,7 +188,7 @@ public class InfoResource
@PathParam("segmentId") String segmentId
)
{
DruidServer server = serverInventoryManager.getInventoryValue(serverName);
DruidServer server = serverInventoryView.getInventoryValue(serverName);
if (server == null) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -217,7 +213,7 @@ public class InfoResource
return builder.entity(
Iterables.concat(
Iterables.transform(
serverInventoryManager.getInventory(),
serverInventoryView.getInventory(),
new Function<DruidServer, Iterable<DataSegment>>()
{
@Override
@ -234,7 +230,7 @@ public class InfoResource
return builder.entity(
Iterables.concat(
Iterables.transform(
serverInventoryManager.getInventory(),
serverInventoryView.getInventory(),
new Function<DruidServer, Iterable<String>>()
{
@Override
@ -265,7 +261,7 @@ public class InfoResource
@PathParam("segmentId") String segmentId
)
{
for (DruidServer server : serverInventoryManager.getInventory()) {
for (DruidServer server : serverInventoryView.getInventory()) {
if (server.getSegments().containsKey(segmentId)) {
return Response.status(Response.Status.OK)
.entity(server.getSegments().get(segmentId))
@ -282,7 +278,7 @@ public class InfoResource
public Response getTiers()
{
Set<String> tiers = Sets.newHashSet();
for (DruidServer server : serverInventoryManager.getInventory()) {
for (DruidServer server : serverInventoryView.getInventory()) {
tiers.add(server.getTier());
}
return Response.status(Response.Status.OK)
@ -376,7 +372,8 @@ public class InfoResource
@QueryParam("interval") final String interval
)
{
// TODO: will likely be all rewritten once Guice introduced
// This is weird enough to have warranted some sort of T0D0 comment at one point, but it will likely be all
// rewritten once Guice introduced, and that's the brunt of the information that was in the original T0D0 too.
if (indexingServiceClient == null) {
return Response.status(Response.Status.OK).entity(ImmutableMap.of("error", "no indexing service found")).build();
}
@ -493,7 +490,7 @@ public class InfoResource
Iterable<DruidDataSource> dataSources =
Iterables.concat(
Iterables.transform(
serverInventoryManager.getInventory(),
serverInventoryView.getInventory(),
new Function<DruidServer, DruidDataSource>()
{
@Override
@ -547,7 +544,7 @@ public class InfoResource
Lists.newArrayList(
Iterables.concat(
Iterables.transform(
serverInventoryManager.getInventory(),
serverInventoryView.getInventory(),
new Function<DruidServer, Iterable<DruidDataSource>>()
{
@Override

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.servlet.GuiceFilter;
@ -31,13 +32,13 @@ import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.config.Config;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.ServerInventoryManager;
import com.metamx.druid.client.ServerInventoryManagerConfig;
import com.metamx.druid.client.ServerInventoryThingieConfig;
import com.metamx.druid.client.ServerInventoryView;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.concurrent.Execs;
import com.metamx.druid.config.ConfigManager;
import com.metamx.druid.config.ConfigManagerConfig;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.coordination.DruidClusterInfo;
import com.metamx.druid.coordination.DruidClusterInfoConfig;
import com.metamx.druid.db.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseRuleManagerConfig;
import com.metamx.druid.db.DatabaseSegmentManager;
@ -47,13 +48,12 @@ import com.metamx.druid.db.DbConnectorConfig;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.initialization.ZkClientConfig;
import com.metamx.druid.initialization.ZkPathsConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.master.DruidMasterConfig;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.master.LoadQueuePeon;
import com.metamx.druid.master.LoadQueueTaskMaster;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters;
@ -67,12 +67,9 @@ import com.metamx.metrics.Monitor;
import com.metamx.metrics.MonitorScheduler;
import com.metamx.metrics.MonitorSchedulerConfig;
import com.metamx.metrics.SysMonitor;
import com.metamx.phonebook.PhoneBook;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.x.discovery.ServiceDiscovery;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.I0Itec.zkclient.ZkClient;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.x.discovery.ServiceDiscovery;
import org.apache.curator.x.discovery.ServiceProvider;
import org.joda.time.Duration;
import org.mortbay.jetty.Server;
import org.mortbay.jetty.servlet.Context;
@ -84,7 +81,8 @@ import org.skife.jdbi.v2.DBI;
import java.net.URL;
import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
/**
@ -102,16 +100,13 @@ public class MasterMain
final ConfigurationObjectFactory configFactory = Config.createFactory(props);
final Lifecycle lifecycle = new Lifecycle();
final HttpClient httpClient = HttpClientInit.createClient(
HttpClientConfig.builder().withNumConnections(1).withReadTimeout(
new Duration(
PropUtils.getProperty(
props,
"druid.emitter.timeOut"
)
)
).build(), lifecycle
);
final HttpClientConfig.Builder httpClientConfigBuilder = HttpClientConfig.builder().withNumConnections(1);
final String emitterTimeout = props.getProperty("druid.emitter.timeOut");
if (emitterTimeout != null) {
httpClientConfigBuilder.withReadTimeout(new Duration(emitterTimeout));
}
final HttpClient httpClient = HttpClientInit.createClient(httpClientConfigBuilder.build(), lifecycle);
final ServiceEmitter emitter = new ServiceEmitter(
PropUtils.getProperty(props, "druid.service"),
@ -120,37 +115,45 @@ public class MasterMain
);
EmittingLogger.registerEmitter(emitter);
final ZkClient zkClient = Initialization.makeZkClient(configFactory.build(ZkClientConfig.class), lifecycle);
final PhoneBook masterYp = Initialization.createPhoneBook(jsonMapper, zkClient, "Master-ZKYP--%s", lifecycle);
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
final ServerInventoryManager serverInventoryManager =
new ServerInventoryManager(configFactory.build(ServerInventoryManagerConfig.class), masterYp);
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
CuratorFramework curatorFramework = Initialization.makeCuratorFramework(
serviceDiscoveryConfig,
lifecycle
);
final ZkPathsConfig zkPaths = configFactory.build(ZkPathsConfig.class);
final ExecutorService exec = Executors.newFixedThreadPool(
1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build()
);
ServerInventoryView serverInventoryView = new ServerInventoryView(
configFactory.build(ServerInventoryThingieConfig.class), zkPaths, curatorFramework, exec, jsonMapper
);
lifecycle.addManagedInstance(serverInventoryView);
final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class);
final DatabaseRuleManagerConfig databaseRuleManagerConfig = configFactory.build(DatabaseRuleManagerConfig.class);
final DBI dbi = new DbConnector(dbConnectorConfig).getDBI();
DbConnector.createSegmentTable(dbi, PropUtils.getProperty(props, "druid.database.segmentTable"));
DbConnector.createRuleTable(dbi, PropUtils.getProperty(props, "druid.database.ruleTable"));
DatabaseRuleManager.createDefaultRule(
dbi, databaseRuleManagerConfig.getRuleTable(), databaseRuleManagerConfig.getDefaultDatasource(), jsonMapper
);
final DatabaseSegmentManager databaseSegmentManager = new DatabaseSegmentManager(
jsonMapper,
scheduledExecutorFactory.create(1, "DatabaseSegmentManager-Exec--%d"),
configFactory.build(DatabaseSegmentManagerConfig.class),
dbi
);
final DatabaseRuleManagerConfig databaseRuleManagerConfig = configFactory.build(DatabaseRuleManagerConfig.class);
final DatabaseRuleManager databaseRuleManager = new DatabaseRuleManager(
jsonMapper,
scheduledExecutorFactory.create(1, "DatabaseRuleManager-Exec--%d"),
databaseRuleManagerConfig,
dbi
);
DatabaseRuleManager.createDefaultRule(
dbi,
databaseRuleManagerConfig.getRuleTable(),
databaseRuleManagerConfig.getDefaultDatasource(),
jsonMapper
);
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
final MonitorScheduler healthMonitor = new MonitorScheduler(
@ -166,12 +169,6 @@ public class MasterMain
final DruidMasterConfig druidMasterConfig = configFactory.build(DruidMasterConfig.class);
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
CuratorFramework curatorFramework = Initialization.makeCuratorFrameworkClient(
serviceDiscoveryConfig,
lifecycle
);
final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient(
curatorFramework,
serviceDiscoveryConfig,
@ -188,29 +185,28 @@ public class MasterMain
indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider);
}
final DruidClusterInfo druidClusterInfo = new DruidClusterInfo(
configFactory.build(DruidClusterInfoConfig.class),
masterYp
);
final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class);
DbConnector.createConfigTable(dbi, configManagerConfig.getConfigTable());
JacksonConfigManager configManager = new JacksonConfigManager(
new ConfigManager(dbi, configManagerConfig), jsonMapper
);
final LoadQueueTaskMaster taskMaster = new LoadQueueTaskMaster(
curatorFramework, jsonMapper, Execs.singleThreaded("Master-PeonExec--%d")
);
final DruidMaster master = new DruidMaster(
druidMasterConfig,
druidClusterInfo,
zkPaths,
configManager,
databaseSegmentManager,
serverInventoryManager,
serverInventoryView,
databaseRuleManager,
masterYp,
curatorFramework,
emitter,
scheduledExecutorFactory,
new ConcurrentHashMap<String, LoadQueuePeon>(),
indexingServiceClient
indexingServiceClient,
taskMaster
);
lifecycle.addManagedInstance(master);
@ -238,10 +234,9 @@ public class MasterMain
final Injector injector = Guice.createInjector(
new MasterServletModule(
serverInventoryManager,
serverInventoryView,
databaseSegmentManager,
databaseRuleManager,
druidClusterInfo,
master,
jsonMapper,
indexingServiceClient
@ -262,9 +257,18 @@ public class MasterMain
public URL getRedirectURL(String queryString, String requestURI)
{
try {
return (queryString == null) ?
new URL(String.format("http://%s%s", druidClusterInfo.getMasterHost(), requestURI)) :
new URL(String.format("http://%s%s?%s", druidClusterInfo.getMasterHost(), requestURI, queryString));
final String currentMaster = master.getCurrentMaster();
if (currentMaster == null) {
return null;
}
String location = String.format("http://%s%s", currentMaster, requestURI);
if (queryString != null) {
location = String.format("%s?%s", location, queryString);
}
return new URL(location);
}
catch (Exception e) {
throw Throwables.propagate(e);

View File

@ -22,12 +22,12 @@ package com.metamx.druid.http;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
import com.google.inject.Provides;
import com.metamx.druid.client.ServerInventoryManager;
import com.metamx.druid.coordination.DruidClusterInfo;
import com.google.inject.util.Providers;
import com.metamx.druid.client.ServerInventoryView;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.db.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseSegmentManager;
import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.sun.jersey.guice.JerseyServletModule;
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
@ -37,28 +37,25 @@ import javax.inject.Singleton;
*/
public class MasterServletModule extends JerseyServletModule
{
private final ServerInventoryManager serverInventoryManager;
private final ServerInventoryView serverInventoryView;
private final DatabaseSegmentManager segmentInventoryManager;
private final DatabaseRuleManager databaseRuleManager;
private final DruidClusterInfo druidClusterInfo;
private final DruidMaster master;
private final ObjectMapper jsonMapper;
private final IndexingServiceClient indexingServiceClient;
public MasterServletModule(
ServerInventoryManager serverInventoryManager,
ServerInventoryView serverInventoryView,
DatabaseSegmentManager segmentInventoryManager,
DatabaseRuleManager databaseRuleManager,
DruidClusterInfo druidClusterInfo,
DruidMaster master,
ObjectMapper jsonMapper,
IndexingServiceClient indexingServiceClient
)
{
this.serverInventoryManager = serverInventoryManager;
this.serverInventoryView = serverInventoryView;
this.segmentInventoryManager = segmentInventoryManager;
this.databaseRuleManager = databaseRuleManager;
this.druidClusterInfo = druidClusterInfo;
this.master = master;
this.jsonMapper = jsonMapper;
this.indexingServiceClient = indexingServiceClient;
@ -69,12 +66,16 @@ public class MasterServletModule extends JerseyServletModule
{
bind(InfoResource.class);
bind(MasterResource.class);
bind(ServerInventoryManager.class).toInstance(serverInventoryManager);
bind(ServerInventoryView.class).toInstance(serverInventoryView);
bind(DatabaseSegmentManager.class).toInstance(segmentInventoryManager);
bind(DatabaseRuleManager.class).toInstance(databaseRuleManager);
bind(DruidMaster.class).toInstance(master);
bind(DruidClusterInfo.class).toInstance(druidClusterInfo);
bind(IndexingServiceClient.class).toInstance(indexingServiceClient);
if (indexingServiceClient == null) {
bind(IndexingServiceClient.class).toProvider(Providers.<IndexingServiceClient>of(null));
}
else {
bind(IndexingServiceClient.class).toInstance(indexingServiceClient);
}
serve("/*").with(GuiceContainer.class);
}

View File

@ -63,11 +63,16 @@ public class RedirectServlet extends DefaultServlet
if (redirectInfo.doLocal()) {
super.service(request, response);
} else {
URL url = redirectInfo.getRedirectURL(request.getQueryString(), request.getRequestURI());
log.info("Forwarding request to [%s]", url);
final URL redirectURL = redirectInfo.getRedirectURL(request.getQueryString(), request.getRequestURI());
if (redirectURL == null) {
response.setStatus(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
}
else {
log.info("Forwarding request to [%s]", redirectURL);
response.setStatus(HttpServletResponse.SC_MOVED_TEMPORARILY);
response.setHeader("Location", url.toString());
response.setStatus(HttpServletResponse.SC_MOVED_TEMPORARILY);
response.setHeader("Location", redirectURL.toString());
}
}
}
}

View File

@ -21,13 +21,13 @@ package com.metamx.druid.master;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.metamx.common.ISE;
import com.google.common.io.Closeables;
import com.metamx.common.Pair;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.concurrent.ScheduledExecutors;
@ -38,20 +38,21 @@ import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.DruidDataSource;
import com.metamx.druid.client.DruidServer;
import com.metamx.druid.client.ServerInventoryManager;
import com.metamx.druid.client.ServerInventoryView;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.concurrent.Execs;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.coordination.DruidClusterInfo;
import com.metamx.druid.db.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseSegmentManager;
import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.initialization.ZkPathsConfig;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import com.metamx.phonebook.PhoneBook;
import com.metamx.phonebook.PhoneBookPeon;
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.leader.LeaderLatch;
import org.apache.curator.framework.recipes.leader.LeaderLatchListener;
import org.apache.curator.utils.ZKPaths;
import org.joda.time.DateTime;
import org.joda.time.Duration;
@ -61,6 +62,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
@ -78,49 +80,81 @@ public class DruidMaster
private volatile boolean master = false;
private final DruidMasterConfig config;
private final DruidClusterInfo clusterInfo;
private final ZkPathsConfig zkPaths;
private final JacksonConfigManager configManager;
private final DatabaseSegmentManager databaseSegmentManager;
private final ServerInventoryManager serverInventoryManager;
private final ServerInventoryView serverInventoryView;
private final DatabaseRuleManager databaseRuleManager;
private final PhoneBook yp;
private final CuratorFramework curator;
private final ServiceEmitter emitter;
private final IndexingServiceClient indexingServiceClient;
private final ScheduledExecutorService exec;
private final ScheduledExecutorService peonExec;
private final PhoneBookPeon masterPeon;
private final LoadQueueTaskMaster taskMaster;
private final Map<String, LoadQueuePeon> loadManagementPeons;
private final AtomicReference<LeaderLatch> leaderLatch;
public DruidMaster(
DruidMasterConfig config,
DruidClusterInfo clusterInfo,
ZkPathsConfig zkPaths,
JacksonConfigManager configManager,
DatabaseSegmentManager databaseSegmentManager,
ServerInventoryManager serverInventoryManager,
ServerInventoryView serverInventoryView,
DatabaseRuleManager databaseRuleManager,
PhoneBook zkPhoneBook,
CuratorFramework curator,
ServiceEmitter emitter,
ScheduledExecutorFactory scheduledExecutorFactory,
Map<String, LoadQueuePeon> loadManagementPeons,
IndexingServiceClient indexingServiceClient
IndexingServiceClient indexingServiceClient,
LoadQueueTaskMaster taskMaster
)
{
this(
config,
zkPaths,
configManager,
databaseSegmentManager,
serverInventoryView,
databaseRuleManager,
curator,
emitter,
scheduledExecutorFactory,
indexingServiceClient,
taskMaster,
Maps.<String, LoadQueuePeon>newConcurrentMap()
);
}
DruidMaster(
DruidMasterConfig config,
ZkPathsConfig zkPaths,
JacksonConfigManager configManager,
DatabaseSegmentManager databaseSegmentManager,
ServerInventoryView serverInventoryView,
DatabaseRuleManager databaseRuleManager,
CuratorFramework curator,
ServiceEmitter emitter,
ScheduledExecutorFactory scheduledExecutorFactory,
IndexingServiceClient indexingServiceClient,
LoadQueueTaskMaster taskMaster,
ConcurrentMap<String, LoadQueuePeon> loadQueuePeonMap
)
{
this.config = config;
this.clusterInfo = clusterInfo;
this.zkPaths = zkPaths;
this.configManager = configManager;
this.databaseSegmentManager = databaseSegmentManager;
this.serverInventoryManager = serverInventoryManager;
this.serverInventoryView = serverInventoryView;
this.databaseRuleManager = databaseRuleManager;
this.yp = zkPhoneBook;
this.curator = curator;
this.emitter = emitter;
this.indexingServiceClient = indexingServiceClient;
this.taskMaster = taskMaster;
this.masterPeon = new MasterListeningPeon();
this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d");
this.peonExec = scheduledExecutorFactory.create(1, "Master-PeonExec--%d");
this.loadManagementPeons = loadManagementPeons;
this.leaderLatch = new AtomicReference<LeaderLatch>(null);
this.loadManagementPeons = loadQueuePeonMap;
}
public boolean isClusterMaster()
@ -143,7 +177,7 @@ public class DruidMaster
// find segments currently loaded
Map<String, Set<DataSegment>> segmentsInCluster = Maps.newHashMap();
for (DruidServer druidServer : serverInventoryManager.getInventory()) {
for (DruidServer druidServer : serverInventoryView.getInventory()) {
for (DruidDataSource druidDataSource : druidServer.getDataSources()) {
Set<DataSegment> segments = segmentsInCluster.get(druidDataSource.getName());
if (segments == null) {
@ -175,12 +209,12 @@ public class DruidMaster
public int lookupSegmentLifetime(DataSegment segment)
{
return serverInventoryManager.lookupSegmentLifetime(segment);
return serverInventoryView.lookupSegmentLifetime(segment);
}
public void decrementRemovedSegmentsLifetime()
{
serverInventoryManager.decrementRemovedSegmentsLifetime();
serverInventoryView.decrementRemovedSegmentsLifetime();
}
public void removeSegment(DataSegment segment)
@ -199,14 +233,25 @@ public class DruidMaster
databaseSegmentManager.enableDatasource(ds);
}
public String getCurrentMaster()
{
try {
final LeaderLatch latch = leaderLatch.get();
return latch == null ? null : latch.getLeader().getId();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
public void moveSegment(String from, String to, String segmentName, final LoadPeonCallback callback)
{
final DruidServer fromServer = serverInventoryManager.getInventoryValue(from);
final DruidServer fromServer = serverInventoryView.getInventoryValue(from);
if (fromServer == null) {
throw new IllegalArgumentException(String.format("Unable to find server [%s]", from));
}
final DruidServer toServer = serverInventoryManager.getInventoryValue(to);
final DruidServer toServer = serverInventoryView.getInventoryValue(to);
if (toServer == null) {
throw new IllegalArgumentException(String.format("Unable to find server [%s]", to));
}
@ -247,8 +292,10 @@ public class DruidMaster
);
}
final String toLoadQueueSegPath = yp.combineParts(Arrays.asList(config.getLoadQueuePath(), to, segmentName));
final String toServedSegPath = yp.combineParts(Arrays.asList(config.getServedSegmentsLocation(), to, segmentName));
final String toLoadQueueSegPath = ZKPaths.makePath(ZKPaths.makePath(zkPaths.getLoadQueuePath(), to), segmentName);
final String toServedSegPath = ZKPaths.makePath(
ZKPaths.makePath(zkPaths.getServedSegmentsPath(), to), segmentName
);
loadPeon.loadSegment(
segment,
@ -257,12 +304,17 @@ public class DruidMaster
@Override
protected void execute()
{
if ((yp.lookup(toServedSegPath, Object.class) != null) &&
yp.lookup(toLoadQueueSegPath, Object.class) == null &&
!dropPeon.getSegmentsToDrop().contains(segment)) {
dropPeon.dropSegment(segment, callback);
} else if (callback != null) {
callback.execute();
try {
if (curator.checkExists().forPath(toServedSegPath) != null &&
curator.checkExists().forPath(toLoadQueueSegPath) == null &&
!dropPeon.getSegmentsToDrop().contains(segment)) {
dropPeon.dropSegment(segment, callback);
} else if (callback != null) {
callback.execute();
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
@ -271,12 +323,12 @@ public class DruidMaster
public void cloneSegment(String from, String to, String segmentName, LoadPeonCallback callback)
{
final DruidServer fromServer = serverInventoryManager.getInventoryValue(from);
final DruidServer fromServer = serverInventoryView.getInventoryValue(from);
if (fromServer == null) {
throw new IllegalArgumentException(String.format("Unable to find server [%s]", from));
}
final DruidServer toServer = serverInventoryManager.getInventoryValue(to);
final DruidServer toServer = serverInventoryView.getInventoryValue(to);
if (toServer == null) {
throw new IllegalArgumentException(String.format("Unable to find server [%s]", to));
}
@ -313,7 +365,7 @@ public class DruidMaster
public void dropSegment(String from, String segmentName, final LoadPeonCallback callback)
{
final DruidServer fromServer = serverInventoryManager.getInventoryValue(from);
final DruidServer fromServer = serverInventoryView.getInventoryValue(from);
if (fromServer == null) {
throw new IllegalArgumentException(String.format("Unable to find server [%s]", from));
}
@ -374,15 +426,43 @@ public class DruidMaster
}
started = true;
if (!yp.isStarted()) {
throw new ISE("Master cannot perform without his yellow pages.");
createNewLeaderLatch();
try {
leaderLatch.get().start();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
becomeMaster();
yp.registerListener(config.getBasePath(), masterPeon);
}
}
private LeaderLatch createNewLeaderLatch()
{
final LeaderLatch newLeaderLatch = new LeaderLatch(
curator, ZKPaths.makePath(zkPaths.getMasterPath(), MASTER_OWNER_NODE), config.getHost()
);
newLeaderLatch.attachListener(
new LeaderLatchListener()
{
@Override
public void becomeMaster()
{
DruidMaster.this.becomeMaster();
}
@Override
public void stopBeingMaster()
{
DruidMaster.this.stopBeingMaster();
}
},
Execs.singleThreaded("MasterLeader-%s")
);
return leaderLatch.getAndSet(newLeaderLatch);
}
@LifecycleStop
public void stop()
{
@ -392,12 +472,10 @@ public class DruidMaster
}
stopBeingMaster();
yp.unregisterListener(config.getBasePath(), masterPeon);
started = false;
exec.shutdownNow();
peonExec.shutdownNow();
}
}
@ -408,41 +486,22 @@ public class DruidMaster
return;
}
boolean becameMaster = true;
log.info("I am the master, all must bow!");
try {
yp.announce(
config.getBasePath(),
MASTER_OWNER_NODE,
ImmutableMap.of(
"host", config.getHost()
)
);
}
catch (ZkNodeExistsException e) {
log.info("Got ZkNodeExistsException, not becoming master.");
becameMaster = false;
}
if (becameMaster) {
log.info("I am the master, all must bow!");
master = true;
databaseSegmentManager.start();
databaseRuleManager.start();
serverInventoryManager.start();
serverInventoryView.start();
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
if (indexingServiceClient != null) {
masterRunnables.add(
Pair.of(
new MasterIndexingServiceRunnable(
makeIndexingServiceHelpers(
configManager.watch(
MergerWhitelist.CONFIG_KEY,
MergerWhitelist.class
)
)
makeIndexingServiceHelpers(configManager.watch(MergerWhitelist.CONFIG_KEY, MergerWhitelist.class))
),
config.getMasterSegmentMergerPeriod()
)
@ -473,11 +532,22 @@ public class DruidMaster
}
);
}
} else {
log.info(
"FAILED to become master!!11!12 Wtfpwned by [%s]",
clusterInfo.lookupCurrentLeader()
);
}
catch (Exception e) {
log.makeAlert(e, "Unable to become master")
.emit();
final LeaderLatch oldLatch = createNewLeaderLatch();
Closeables.closeQuietly(oldLatch);
try {
leaderLatch.get().start();
}
catch (Exception e1) {
// If an exception gets thrown out here, then the master will zombie out 'cause it won't be looking for
// the latch anymore. I don't believe it's actually possible for an Exception to throw out here, but
// Curator likes to have "throws Exception" on methods so it might happen...
log.makeAlert(e1, "I am a zombie")
.emit();
}
}
}
}
@ -485,26 +555,24 @@ public class DruidMaster
private void stopBeingMaster()
{
synchronized (lock) {
log.debug("I am %s the master", master ? "DEFINITELY" : "NOT");
if (master) {
try {
log.info("I am no longer the master...");
leaderLatch.get().close();
for (String server : loadManagementPeons.keySet()) {
LoadQueuePeon peon = loadManagementPeons.remove(server);
peon.stop();
yp.unregisterListener(
yp.combineParts(Arrays.asList(config.getLoadQueuePath(), server)),
peon
);
}
loadManagementPeons.clear();
yp.unannounce(config.getBasePath(), MASTER_OWNER_NODE);
databaseSegmentManager.stop();
serverInventoryManager.stop();
serverInventoryView.stop();
master = false;
}
catch (Exception e) {
log.makeAlert(e, "Unable to stopBeingMaster").emit();
}
}
}
@ -577,35 +645,6 @@ public class DruidMaster
}
}
private class MasterListeningPeon implements PhoneBookPeon<Map>
{
@Override
public Class<Map> getObjectClazz()
{
return Map.class;
}
@Override
public void newEntry(String name, Map properties)
{
if (MASTER_OWNER_NODE.equals(name)) {
if (config.getHost().equals(properties.get("host"))) {
log.info("I really am the master!");
} else {
log.info("[%s] is the real master...", properties);
}
}
}
@Override
public void entryRemoved(String name)
{
if (MASTER_OWNER_NODE.equals(name)) {
becomeMaster();
}
}
}
public abstract class MasterRunnable implements Runnable
{
private final long startTime = System.currentTimeMillis();
@ -621,9 +660,9 @@ public class DruidMaster
{
try {
synchronized (lock) {
Map<String, String> currLeader = clusterInfo.lookupCurrentLeader();
if (currLeader == null || !config.getHost().equals(currLeader.get("host"))) {
log.info("I thought I was the master, but really [%s] is. Phooey.", currLeader);
final LeaderLatch latch = leaderLatch.get();
if (latch == null || !latch.hasLeadership()) {
log.info("[%s] is master, not me. Phooey.", latch == null ? null : latch.getLeader().getId());
stopBeingMaster();
return;
}
@ -631,7 +670,7 @@ public class DruidMaster
List<Boolean> allStarted = Arrays.asList(
databaseSegmentManager.isStarted(),
serverInventoryManager.isStarted()
serverInventoryView.isStarted()
);
for (Boolean aBoolean : allStarted) {
if (!aBoolean) {
@ -678,7 +717,7 @@ public class DruidMaster
{
// Display info about all historical servers
Iterable<DruidServer> servers = FunctionalIterable
.create(serverInventoryManager.getInventory())
.create(serverInventoryView.getInventory())
.filter(
new Predicate<DruidServer>()
{
@ -691,6 +730,7 @@ public class DruidMaster
}
}
);
if (log.isDebugEnabled()) {
log.debug("Servers");
for (DruidServer druidServer : servers) {
@ -706,15 +746,11 @@ public class DruidMaster
final DruidCluster cluster = new DruidCluster();
for (DruidServer server : servers) {
if (!loadManagementPeons.containsKey(server.getName())) {
String basePath = yp.combineParts(Arrays.asList(config.getLoadQueuePath(), server.getName()));
LoadQueuePeon loadQueuePeon = new LoadQueuePeon(yp, basePath, peonExec);
String basePath = ZKPaths.makePath(zkPaths.getLoadQueuePath(), server.getName());
LoadQueuePeon loadQueuePeon = taskMaster.giveMePeon(basePath);
log.info("Creating LoadQueuePeon for server[%s] at path[%s]", server.getName(), basePath);
loadManagementPeons.put(
server.getName(),
loadQueuePeon
);
yp.registerListener(basePath, loadQueuePeon);
loadManagementPeons.put(server.getName(), loadQueuePeon);
}
cluster.add(new ServerHolder(server, loadManagementPeons.get(server.getName())));
@ -741,8 +777,6 @@ public class DruidMaster
log.info("Removing listener for server[%s] which is no longer there.", name);
LoadQueuePeon peon = loadManagementPeons.remove(name);
peon.stop();
yp.unregisterListener(yp.combineParts(Arrays.asList(config.getLoadQueuePath(), name)), peon);
}
decrementRemovedSegmentsLifetime();

View File

@ -67,7 +67,7 @@ public class DruidMasterBalancer implements DruidMasterHelper
if (holder.getLifetime() <= 0) {
log.makeAlert("[%s]: Balancer move segments queue has a segment stuck", tier)
.addData("segment", holder.getSegment().getIdentifier())
.addData("server", holder.getFromServer().getStringProps())
.addData("server", holder.getFromServer().getMetadata())
.emit();
}
}

View File

@ -30,15 +30,6 @@ public abstract class DruidMasterConfig
@Config("druid.host")
public abstract String getHost();
@Config("druid.zk.paths.masterPath")
public abstract String getBasePath();
@Config("druid.zk.paths.loadQueuePath")
public abstract String getLoadQueuePath();
@Config("druid.zk.paths.servedSegmentsPath")
public abstract String getServedSegmentsLocation();
@Config("druid.master.startDelay")
@Default("PT600s")
public abstract Duration getMasterStartDelay();

View File

@ -19,43 +19,47 @@
package com.metamx.druid.master;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.collect.Collections2;
import com.google.common.collect.Lists;
import com.metamx.common.guava.Comparators;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.coordination.DataSegmentChangeRequest;
import com.metamx.druid.coordination.SegmentChangeRequestDrop;
import com.metamx.druid.coordination.SegmentChangeRequestLoad;
import com.metamx.phonebook.PhoneBook;
import com.metamx.phonebook.PhoneBookPeon;
import com.metamx.druid.coordination.SegmentChangeRequestNoop;
import com.metamx.emitter.EmittingLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.CuratorWatcher;
import org.apache.curator.utils.ZKPaths;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.data.Stat;
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicLong;
/**
*/
public class LoadQueuePeon implements PhoneBookPeon<Map>
public class LoadQueuePeon
{
private static final Logger log = new Logger(LoadQueuePeon.class);
private static final EmittingLogger log = new EmittingLogger(LoadQueuePeon.class);
private static final int DROP = 0;
private static final int LOAD = 1;
private final Object lock = new Object();
private final PhoneBook yp;
private final CuratorFramework curator;
private final String basePath;
private final ScheduledExecutorService zkWritingExecutor;
private final ObjectMapper jsonMapper;
private final ExecutorService zkWritingExecutor;
private final AtomicLong queuedSize = new AtomicLong(0);
@ -80,72 +84,18 @@ public class LoadQueuePeon implements PhoneBookPeon<Map>
private volatile SegmentHolder currentlyLoading = null;
LoadQueuePeon(
PhoneBook yp,
CuratorFramework curator,
String basePath,
ScheduledExecutorService zkWritingExecutor
ObjectMapper jsonMapper,
ExecutorService zkWritingExecutor
)
{
this.yp = yp;
this.curator = curator;
this.basePath = basePath;
this.jsonMapper = jsonMapper;
this.zkWritingExecutor = zkWritingExecutor;
}
@Override
public Class<Map> getObjectClazz()
{
return Map.class;
}
@Override
public void newEntry(String name, Map properties)
{
synchronized (lock) {
if (currentlyLoading == null) {
log.warn(
"Server[%s] a new entry[%s] appeared, even though nothing is currently loading[%s]",
basePath,
name,
currentlyLoading
);
} else {
if (!name.equals(currentlyLoading.getSegmentIdentifier())) {
log.warn(
"Server[%s] a new entry[%s] appeared that is not the currently loading entry[%s]",
basePath,
name,
currentlyLoading
);
} else {
log.info("Server[%s]'s currently loading entry[%s] appeared.", basePath, name);
}
}
}
}
@Override
public void entryRemoved(String name)
{
synchronized (lock) {
if (currentlyLoading == null) {
log.warn("Server[%s] an entry[%s] was removed even though it wasn't loading!?", basePath, name);
return;
}
if (!name.equals(currentlyLoading.getSegmentIdentifier())) {
log.warn(
"Server[%s] entry [%s] was removed even though it's not what is currently loading[%s]",
basePath,
name,
currentlyLoading
);
return;
}
actionCompleted();
log.info("Server[%s] done processing [%s]", basePath, name);
}
doNext();
}
public Set<DataSegment> getSegmentsToLoad()
{
return new ConcurrentSkipListSet<DataSegment>(
@ -262,7 +212,71 @@ public class LoadQueuePeon implements PhoneBookPeon<Map>
return;
}
submitExecutable();
zkWritingExecutor.execute(
new Runnable()
{
@Override
public void run()
{
synchronized (lock) {
try {
if (currentlyLoading == null) {
log.makeAlert("Crazy race condition! server[%s]", basePath)
.emit();
actionCompleted();
doNext();
return;
}
log.info("Server[%s] adding segment[%s]", basePath, currentlyLoading.getSegmentIdentifier());
final String path = ZKPaths.makePath(basePath, currentlyLoading.getSegmentIdentifier());
final byte[] payload = jsonMapper.writeValueAsBytes(currentlyLoading.getChangeRequest());
curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload);
final Stat stat = curator.checkExists().usingWatcher(
new CuratorWatcher()
{
@Override
public void process(WatchedEvent watchedEvent) throws Exception
{
switch (watchedEvent.getType()) {
case NodeDeleted:
entryRemoved(watchedEvent.getPath());
}
}
}
).forPath(path);
if (stat == null) {
final byte[] noopPayload = jsonMapper.writeValueAsBytes(new SegmentChangeRequestNoop());
// Create a node and then delete it to remove the registered watcher. This is a work-around for
// a zookeeper race condition. Specifically, when you set a watcher, it fires on the next event
// that happens for that node. If no events happen, the watcher stays registered foreverz.
// Couple that with the fact that you cannot set a watcher when you create a node, but what we
// want is to create a node and then watch for it to get deleted. The solution is that you *can*
// set a watcher when you check to see if it exists so, we first create the node and then set a
// watcher on its existence. However, if already does not exist by the time the existence check
// returns, then the watcher that was set will never fire (nobody will ever create the node
// again) and thus lead to a slow, but real, memory leak. So, we create another node to cause
// that watcher to fire and delete it right away.
//
// We do not create the existence watcher first, because then it will fire when we create the
// node and we'll have the same race when trying to refresh that watcher.
curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, noopPayload);
entryRemoved(path);
}
}
catch (Exception e) {
log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, currentlyLoading);
// Act like it was completed so that the master gives it to someone else
actionCompleted();
doNext();
}
}
}
}
);
} else {
log.info(
"Server[%s] skipping doNext() because something is currently loading[%s].", basePath, currentlyLoading
@ -271,74 +285,6 @@ public class LoadQueuePeon implements PhoneBookPeon<Map>
}
}
private void submitExecutable()
{
final SegmentHolder currentlyLoadingRef = currentlyLoading;
final AtomicBoolean postedEphemeral = new AtomicBoolean(false);
zkWritingExecutor.execute(
new Runnable()
{
@Override
public void run()
{
synchronized (lock) {
try {
if (currentlyLoading == null) {
log.error("Crazy race condition! server[%s]", basePath);
postedEphemeral.set(true);
actionCompleted();
doNext();
return;
}
log.info("Server[%s] adding segment[%s]", basePath, currentlyLoading.getSegmentIdentifier());
yp.postEphemeral(
basePath,
currentlyLoading.getSegmentIdentifier(),
currentlyLoading.getChangeRequest()
);
postedEphemeral.set(true);
}
catch (Throwable e) {
log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, currentlyLoading);
// Act like it was completed so that the master gives it to someone else
postedEphemeral.set(true);
actionCompleted();
doNext();
}
}
}
}
);
zkWritingExecutor.schedule(
new Runnable()
{
@Override
public void run()
{
synchronized (lock) {
String path = yp.combineParts(Arrays.asList(basePath, currentlyLoadingRef.getSegmentIdentifier()));
if (!postedEphemeral.get()) {
log.info("Ephemeral hasn't been posted yet for [%s], rescheduling.", path);
zkWritingExecutor.schedule(this, 60, TimeUnit.SECONDS);
}
if (currentlyLoadingRef == currentlyLoading) {
if (yp.lookup(path, Object.class) == null) {
log.info("Looks like [%s] was created and deleted without the watchers finding out.", path);
entryRemoved(currentlyLoadingRef.getSegmentIdentifier());
} else {
log.info("Path[%s] still out on ZK, rescheduling.", path);
zkWritingExecutor.schedule(this, 60, TimeUnit.SECONDS);
}
}
}
}
},
60,
TimeUnit.SECONDS
);
}
private void actionCompleted()
{
if (currentlyLoading != null) {
@ -384,6 +330,27 @@ public class LoadQueuePeon implements PhoneBookPeon<Map>
}
}
private void entryRemoved(String path)
{
synchronized (lock) {
if (currentlyLoading == null) {
log.warn("Server[%s] an entry[%s] was removed even though it wasn't loading!?", basePath, path);
return;
}
if (!ZKPaths.getNodeFromPath(path).equals(currentlyLoading.getSegmentIdentifier())) {
log.warn(
"Server[%s] entry [%s] was removed even though it's not what is currently loading[%s]",
basePath, path, currentlyLoading
);
return;
}
actionCompleted();
log.info("Server[%s] done processing [%s]", basePath, path);
}
doNext();
}
private class SegmentHolder
{
private final DataSegment segment;

View File

@ -17,30 +17,35 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid;
package com.metamx.druid.master;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import org.apache.curator.framework.CuratorFramework;
import org.skife.config.ConfigurationObjectFactory;
import java.util.Properties;
import java.util.concurrent.ExecutorService;
/**
* Provides LoadQueuePeons
*/
@Deprecated
public abstract class BaseNode<T extends BaseNode> extends QueryableNode
public class LoadQueueTaskMaster
{
protected BaseNode(
Logger log,
Properties props,
Lifecycle lifecycle,
private final CuratorFramework curator;
private final ObjectMapper jsonMapper;
private final ExecutorService peonExec;
public LoadQueueTaskMaster(
CuratorFramework curator,
ObjectMapper jsonMapper,
ObjectMapper smileMapper,
ConfigurationObjectFactory configFactory
ExecutorService peonExec
)
{
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
this.curator = curator;
this.jsonMapper = jsonMapper;
this.peonExec = peonExec;
}
public LoadQueuePeon giveMePeon(String basePath)
{
return new LoadQueuePeon(curator, basePath, jsonMapper, peonExec);
}
}

Some files were not shown because too many files have changed in this diff Show More