mirror of https://github.com/apache/druid.git
Merge pull request #131 from metamx/curator
Remove references to i0tec.zkClient and replace with Curator
This commit is contained in:
commit
cd535fcd79
|
@ -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>
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
)
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -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 + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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
|
||||
);
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
|
|
@ -75,6 +75,12 @@ public abstract class AbstractTask implements Task
|
|||
return groupId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getNodeType()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getDataSource()
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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));
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
)
|
||||
);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
39
pom.xml
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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>
|
||||
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
|
@ -40,7 +40,6 @@ public class SegmentChangeRequestDrop implements DataSegmentChangeRequest
|
|||
|
||||
@JsonProperty
|
||||
@JsonUnwrapped
|
||||
@Override
|
||||
public DataSegment getSegment()
|
||||
{
|
||||
return segment;
|
||||
|
|
|
@ -46,7 +46,6 @@ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest
|
|||
|
||||
@JsonProperty
|
||||
@JsonUnwrapped
|
||||
@Override
|
||||
public DataSegment getSegment()
|
||||
{
|
||||
return segment;
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue