Merge branch 'master' into guice

Conflicts:
	client/src/main/java/com/metamx/druid/QueryableNode.java
	client/src/main/java/com/metamx/druid/client/ServerInventoryView.java
	client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java
	client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java
	client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java
	indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java
	indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java
	indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java
	indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java
	indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java
	pom.xml
	server/src/main/java/com/metamx/druid/coordination/ServerManager.java
	server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java
	server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java
	server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java
	server/src/main/java/com/metamx/druid/http/ComputeNode.java
	server/src/main/java/com/metamx/druid/http/MasterMain.java
	server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java
	server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java
	server/src/main/java/com/metamx/druid/master/DruidMaster.java
This commit is contained in:
cheddar 2013-08-01 16:42:47 -07:00
commit 9e78bb38f5
176 changed files with 7314 additions and 1532 deletions

1
.gitignore vendored
View File

@ -13,3 +13,4 @@ target
examples/rand/RealtimeNode.out
examples/twitter/RealtimeNode.out
*.log
*.DS_Store

View File

@ -10,8 +10,6 @@ SCRIPT_DIR=`pwd`
popd
VERSION=`cat pom.xml | grep version | head -4 | tail -1 | sed 's_.*<version>\([^<]*\)</version>.*_\1_'`
#TAR_FILE=${SCRIPT_DIR}/${PROJECT}-${VERSION}.tar.gz
#rm -f ${TAR_FILE}
echo Using Version[${VERSION}]

View File

@ -18,8 +18,7 @@
~ 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>
@ -29,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.5.0-SNAPSHOT</version>
<version>0.5.23-SNAPSHOT</version>
</parent>
<dependencies>
@ -69,6 +68,10 @@
<groupId>commons-codec</groupId>
<artifactId>commons-codec</artifactId>
</dependency>
<dependency>
<groupId>commons-httpclient</groupId>
<artifactId>commons-httpclient</artifactId>
</dependency>
<dependency>
<groupId>org.skife.config</groupId>
<artifactId>config-magic</artifactId>
@ -186,6 +189,10 @@
<groupId>com.metamx</groupId>
<artifactId>bytebuffer-collections</artifactId>
</dependency>
<dependency>
<groupId>net.jpountz.lz4</groupId>
<artifactId>lz4</artifactId>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -25,6 +25,8 @@ 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.IAE;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.concurrent.ScheduledExecutors;
@ -32,21 +34,30 @@ 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.BatchServerInventoryView;
import com.metamx.druid.client.DruidServerConfig;
import com.metamx.druid.client.InventoryView;
import com.metamx.druid.client.ServerInventoryView;
import com.metamx.druid.client.ServerInventoryViewConfig;
import com.metamx.druid.client.ServerView;
import com.metamx.druid.client.SingleServerInventoryView;
import com.metamx.druid.concurrent.Execs;
import com.metamx.druid.coordination.CuratorDataSegmentAnnouncer;
import com.metamx.druid.coordination.AbstractDataSegmentAnnouncer;
import com.metamx.druid.coordination.BatchDataSegmentAnnouncer;
import com.metamx.druid.coordination.DataSegmentAnnouncer;
import com.metamx.druid.coordination.DruidServerMetadata;
import com.metamx.druid.curator.CuratorConfig;
import com.metamx.druid.coordination.MultipleDataSegmentAnnouncerDataSegmentAnnouncer;
import com.metamx.druid.coordination.SingleDataSegmentAnnouncer;
import com.metamx.druid.curator.announcement.Announcer;
import com.metamx.druid.guice.JsonConfigurator;
import com.metamx.druid.http.log.RequestLogger;
import com.metamx.druid.http.NoopRequestLogger;
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.ZkDataSegmentAnnouncerConfig;
import com.metamx.druid.initialization.ZkPathsConfig;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
@ -356,12 +367,32 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
private void initializeServerInventoryView()
{
if (serverInventoryView == null) {
serverInventoryView = new ServerInventoryView(
getConfigFactory().build(ServerInventoryViewConfig.class),
getZkPaths(),
getCuratorFramework(),
getJsonMapper()
final ExecutorService exec = Executors.newFixedThreadPool(
1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build()
);
final ServerInventoryViewConfig serverInventoryViewConfig = getConfigFactory().build(ServerInventoryViewConfig.class);
final String announcerType = serverInventoryViewConfig.getAnnouncerType();
if ("legacy".equalsIgnoreCase(announcerType)) {
serverInventoryView = new SingleServerInventoryView(
serverInventoryViewConfig,
getZkPaths(),
getCuratorFramework(),
exec,
getJsonMapper()
);
} else if ("batch".equalsIgnoreCase(announcerType)) {
serverInventoryView = new BatchServerInventoryView(
serverInventoryViewConfig,
getZkPaths(),
getCuratorFramework(),
exec,
getJsonMapper()
);
} else {
throw new IAE("Unknown type %s", announcerType);
}
lifecycle.addManagedInstance(serverInventoryView);
}
}
@ -371,13 +402,23 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
if (requestLogger == null) {
try {
final String loggingType = props.getProperty("druid.request.logging.type");
if("emitter".equals(loggingType)) {
setRequestLogger(Initialization.makeEmittingRequestLogger(getProps(), getEmitter()));
}
else {
if ("emitter".equals(loggingType)) {
setRequestLogger(
Initialization.makeFileRequestLogger(getJsonMapper(), getScheduledExecutorFactory(), getProps())
Initialization.makeEmittingRequestLogger(
getProps(),
getEmitter()
)
);
} else if ("file".equalsIgnoreCase(loggingType)) {
setRequestLogger(
Initialization.makeFileRequestLogger(
getJsonMapper(),
getScheduledExecutorFactory(),
getProps()
)
);
} else {
setRequestLogger(new NoopRequestLogger());
}
}
catch (IOException e) {
@ -419,7 +460,40 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
final Announcer announcer = new Announcer(getCuratorFramework(), Execs.singleThreaded("Announcer-%s"));
lifecycle.addManagedInstance(announcer);
setAnnouncer(new CuratorDataSegmentAnnouncer(getDruidServerMetadata(), getZkPaths(), announcer, getJsonMapper()));
final ZkDataSegmentAnnouncerConfig config = getConfigFactory().build(ZkDataSegmentAnnouncerConfig.class);
final String announcerType = config.getAnnouncerType();
final DataSegmentAnnouncer dataSegmentAnnouncer;
if ("batch".equalsIgnoreCase(announcerType)) {
dataSegmentAnnouncer = new BatchDataSegmentAnnouncer(
getDruidServerMetadata(),
config,
announcer,
getJsonMapper()
);
} else if ("legacy".equalsIgnoreCase(announcerType)) {
dataSegmentAnnouncer = new MultipleDataSegmentAnnouncerDataSegmentAnnouncer(
Arrays.<AbstractDataSegmentAnnouncer>asList(
new BatchDataSegmentAnnouncer(
getDruidServerMetadata(),
config,
announcer,
getJsonMapper()
),
new SingleDataSegmentAnnouncer(
getDruidServerMetadata(),
getZkPaths(),
announcer,
getJsonMapper()
)
)
);
} else {
throw new ISE("Unknown announcer type [%s]", announcerType);
}
setAnnouncer(dataSegmentAnnouncer);
lifecycle.addManagedInstance(getAnnouncer(), Lifecycle.Stage.LAST);
}
}

View File

@ -0,0 +1,129 @@
/*
* 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.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.MapMaker;
import com.google.common.collect.Sets;
import com.metamx.common.ISE;
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.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
/**
*/
public class BatchServerInventoryView extends ServerInventoryView<Set<DataSegment>>
{
private static final EmittingLogger log = new EmittingLogger(BatchServerInventoryView.class);
final ConcurrentMap<String, Set<DataSegment>> zNodes = new MapMaker().makeMap();
public BatchServerInventoryView(
final ServerInventoryViewConfig config,
final ZkPathsConfig zkPaths,
final CuratorFramework curator,
final ExecutorService exec,
final ObjectMapper jsonMapper
)
{
super(
config,
log,
new InventoryManagerConfig()
{
@Override
public String getContainerPath()
{
return zkPaths.getAnnouncementsPath();
}
@Override
public String getInventoryPath()
{
return zkPaths.getLiveSegmentsPath();
}
},
curator,
exec,
jsonMapper,
new TypeReference<Set<DataSegment>>()
{
}
);
}
@Override
protected DruidServer addInnerInventory(
final DruidServer container,
String inventoryKey,
final Set<DataSegment> inventory
)
{
zNodes.put(inventoryKey, inventory);
for (DataSegment segment : inventory) {
addSingleInventory(container, segment);
}
return container;
}
@Override
protected DruidServer updateInnerInventory(
DruidServer container, String inventoryKey, Set<DataSegment> inventory
)
{
Set<DataSegment> existing = zNodes.get(inventoryKey);
if (existing == null) {
throw new ISE("Trying to update an inventoryKey[%s] that didn't exist?!", inventoryKey);
}
for (DataSegment segment : Sets.difference(inventory, existing)) {
addSingleInventory(container, segment);
}
for (DataSegment segment : Sets.difference(existing, inventory)) {
removeSingleInventory(container, segment.getIdentifier());
}
zNodes.put(inventoryKey, inventory);
return container;
}
@Override
protected DruidServer removeInnerInventory(final DruidServer container, String inventoryKey)
{
log.info("Server[%s] removed container[%s]", container.getName(), inventoryKey);
Set<DataSegment> segments = zNodes.remove(inventoryKey);
if (segments == null) {
log.warn("Told to remove container[%s], which didn't exist", inventoryKey);
return container;
}
for (DataSegment segment : segments) {
removeSingleInventory(container, segment.getIdentifier());
}
return container;
}
}

View File

@ -49,6 +49,7 @@ import com.metamx.druid.client.selector.ServerSelector;
import com.metamx.druid.partition.PartitionChunk;
import com.metamx.druid.query.CacheStrategy;
import com.metamx.druid.query.MetricManipulationFn;
import com.metamx.druid.query.Queries;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.query.QueryToolChest;
import com.metamx.druid.query.QueryToolChestWarehouse;
@ -125,12 +126,18 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
&& strategy != null;
final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment", "false"));
final Query<T> rewrittenQuery;
ImmutableMap.Builder<String, String> contextBuilder = new ImmutableMap.Builder<String, String>();
final String priority = query.getContextValue("priority", "0");
contextBuilder.put("priority", priority);
if (populateCache) {
rewrittenQuery = query.withOverriddenContext(ImmutableMap.of("bySegment", "true", "intermediate", "true"));
} else {
rewrittenQuery = query.withOverriddenContext(ImmutableMap.of("intermediate", "true"));
contextBuilder.put("bySegment", "true");
}
contextBuilder.put("intermediate", "true");
final Query<T> rewrittenQuery = query.withOverriddenContext(contextBuilder.build());
VersionedIntervalTimeline<String, ServerSelector> timeline = serverView.getTimeline(query.getDataSource());
if (timeline == null) {

View File

@ -26,6 +26,7 @@ import com.metamx.common.logger.Logger;
import com.metamx.druid.coordination.DruidServerMetadata;
import com.metamx.druid.initialization.DruidNode;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@ -118,7 +119,8 @@ public class DruidServer implements Comparable
@JsonProperty
public Map<String, DataSegment> getSegments()
{
return ImmutableMap.copyOf(segments);
// Copying the map slows things down a lot here, don't use Immutable Map here
return Collections.unmodifiableMap(segments);
}
public DataSegment getSegment(String segmentName)

View File

@ -20,6 +20,7 @@
package com.metamx.druid.client;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
@ -28,6 +29,7 @@ import com.google.inject.Inject;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.concurrent.Execs;
import com.metamx.common.logger.Logger;
import com.metamx.druid.curator.inventory.CuratorInventoryManager;
import com.metamx.druid.curator.inventory.CuratorInventoryManagerStrategy;
import com.metamx.druid.curator.inventory.InventoryManagerConfig;
@ -45,28 +47,32 @@ import java.util.concurrent.atomic.AtomicBoolean;
/**
*/
@ManageLifecycle
public class ServerInventoryView implements ServerView, InventoryView
public abstract class ServerInventoryView<InventoryType> implements ServerView, InventoryView
{
private static final EmittingLogger log = new EmittingLogger(ServerInventoryView.class);
private final CuratorInventoryManager<DruidServer, DataSegment> inventoryManager;
private final ServerInventoryViewConfig config;
private final Logger log;
private final CuratorInventoryManager<DruidServer, InventoryType> 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();
private final Map<String, Integer> removedSegments = new MapMaker().makeMap();
@Inject
public ServerInventoryView(
final ServerInventoryViewConfig config,
final ZkPathsConfig zkPaths,
final Logger log,
final InventoryManagerConfig inventoryManagerConfig,
final CuratorFramework curator,
final ObjectMapper jsonMapper
final ExecutorService exec,
final ObjectMapper jsonMapper,
final TypeReference<InventoryType> typeReference
)
{
inventoryManager = new CuratorInventoryManager<DruidServer, DataSegment>(
this.config = config;
this.log = log;
this.inventoryManager = new CuratorInventoryManager<DruidServer, InventoryType>(
curator,
new InventoryManagerConfig()
{
@ -108,10 +114,10 @@ public class ServerInventoryView implements ServerView, InventoryView
}
@Override
public DataSegment deserializeInventory(byte[] bytes)
public InventoryType deserializeInventory(byte[] bytes)
{
try {
return jsonMapper.readValue(bytes, DataSegment.class);
return jsonMapper.readValue(bytes, typeReference);
}
catch (IOException e) {
throw Throwables.propagate(e);
@ -119,7 +125,7 @@ public class ServerInventoryView implements ServerView, InventoryView
}
@Override
public byte[] serializeInventory(DataSegment inventory)
public byte[] serializeInventory(InventoryType inventory)
{
try {
return jsonMapper.writeValueAsBytes(inventory);
@ -149,67 +155,27 @@ public class ServerInventoryView implements ServerView, InventoryView
}
@Override
public DruidServer addInventory(final DruidServer container, String inventoryKey, final DataSegment inventory)
public DruidServer addInventory(
final DruidServer container,
String inventoryKey,
final InventoryType inventory
)
{
log.info("Server[%s] added segment[%s]", container.getName(), inventoryKey);
return addInnerInventory(container, inventoryKey, inventory);
}
if (container.getSegment(inventoryKey) != null) {
log.warn(
"Not adding or running callbacks for existing segment[%s] on server[%s]",
inventoryKey,
container.getName()
);
return container;
}
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 updateInventory(
DruidServer container, String inventoryKey, InventoryType inventory
)
{
return updateInnerInventory(container, inventoryKey, inventory);
}
@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);
if (segment == null) {
log.warn(
"Not running cleanup or callbacks for non-existing segment[%s] on server[%s]",
inventoryKey,
container.getName()
);
return container;
}
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;
return removeInnerInventory(container, inventoryKey);
}
}
);
@ -285,7 +251,12 @@ public class ServerInventoryView implements ServerView, InventoryView
segmentCallbacks.put(callback, exec);
}
private void runSegmentCallbacks(
public InventoryManagerConfig getInventoryManagerConfig()
{
return inventoryManager.getConfig();
}
protected void runSegmentCallbacks(
final Function<SegmentCallback, CallbackAction> fn
)
{
@ -305,7 +276,7 @@ public class ServerInventoryView implements ServerView, InventoryView
}
}
private void runServerCallbacks(final DruidServer server)
protected void runServerCallbacks(final DruidServer server)
{
for (final Map.Entry<ServerCallback, Executor> entry : serverCallbacks.entrySet()) {
entry.getValue().execute(
@ -322,4 +293,83 @@ public class ServerInventoryView implements ServerView, InventoryView
);
}
}
protected void addSingleInventory(
final DruidServer container,
final DataSegment inventory
)
{
log.info("Server[%s] added segment[%s]", container.getName(), inventory.getIdentifier());
if (container.getSegment(inventory.getIdentifier()) != null) {
log.warn(
"Not adding or running callbacks for existing segment[%s] on server[%s]",
inventory.getIdentifier(),
container.getName()
);
return;
}
container.addDataSegment(inventory.getIdentifier(), inventory);
runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(SegmentCallback input)
{
return input.segmentAdded(container, inventory);
}
}
);
}
protected void removeSingleInventory(final DruidServer container, String inventoryKey)
{
log.info("Server[%s] removed segment[%s]", container.getName(), inventoryKey);
final DataSegment segment = container.getSegment(inventoryKey);
if (segment == null) {
log.warn(
"Not running cleanup or callbacks for non-existing segment[%s] on server[%s]",
inventoryKey,
container.getName()
);
return;
}
container.removeDataSegment(inventoryKey);
runSegmentCallbacks(
new Function<SegmentCallback, CallbackAction>()
{
@Override
public CallbackAction apply(SegmentCallback input)
{
return input.segmentRemoved(container, segment);
}
}
);
removedSegments.put(inventoryKey, config.getRemovedSegmentLifetime());
}
protected abstract DruidServer addInnerInventory(
final DruidServer container,
String inventoryKey,
final InventoryType inventory
);
protected abstract DruidServer updateInnerInventory(
final DruidServer container,
String inventoryKey,
final InventoryType inventory
);
protected abstract DruidServer removeInnerInventory(
final DruidServer container,
String inventoryKey
);
}

View File

@ -29,4 +29,8 @@ public abstract class ServerInventoryViewConfig
@Config("druid.master.removedSegmentLifetime")
@Default("1")
public abstract int getRemovedSegmentLifetime();
@Config("druid.announcer.type")
@Default("legacy")
public abstract String getAnnouncerType();
}

View File

@ -0,0 +1,94 @@
/*
* 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.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
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.util.concurrent.ExecutorService;
/**
*/
public class SingleServerInventoryView extends ServerInventoryView<DataSegment>
{
private static final EmittingLogger log = new EmittingLogger(SingleServerInventoryView.class);
public SingleServerInventoryView(
final ServerInventoryViewConfig config,
final ZkPathsConfig zkPaths,
final CuratorFramework curator,
final ExecutorService exec,
final ObjectMapper jsonMapper
)
{
super(
config,
log,
new InventoryManagerConfig()
{
@Override
public String getContainerPath()
{
return zkPaths.getAnnouncementsPath();
}
@Override
public String getInventoryPath()
{
return zkPaths.getServedSegmentsPath();
}
},
curator,
exec,
jsonMapper,
new TypeReference<DataSegment>()
{
}
);
}
@Override
protected DruidServer addInnerInventory(
DruidServer container, String inventoryKey, DataSegment inventory
)
{
addSingleInventory(container, inventory);
return container;
}
@Override
protected DruidServer updateInnerInventory(
DruidServer container, String inventoryKey, DataSegment inventory
)
{
return addInnerInventory(container, inventoryKey, inventory);
}
@Override
protected DruidServer removeInnerInventory(DruidServer container, String inventoryKey)
{
removeSingleInventory(container, inventoryKey);
return container;
}
}

View File

@ -0,0 +1,81 @@
/*
* Druid - a distributed column store.
* Copyright (C) 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.client.cache;
import com.google.common.primitives.Ints;
import net.jpountz.lz4.LZ4Compressor;
import net.jpountz.lz4.LZ4Decompressor;
import net.jpountz.lz4.LZ4Factory;
import net.spy.memcached.transcoders.SerializingTranscoder;
import java.nio.ByteBuffer;
public class LZ4Transcoder extends SerializingTranscoder
{
private final LZ4Factory lz4Factory;
public LZ4Transcoder()
{
super();
lz4Factory = LZ4Factory.fastestJavaInstance();
}
public LZ4Transcoder(int max)
{
super(max);
lz4Factory = LZ4Factory.fastestJavaInstance();
}
@Override
protected byte[] compress(byte[] in)
{
if (in == null) {
throw new NullPointerException("Can't compress null");
}
LZ4Compressor compressor = lz4Factory.fastCompressor();
byte[] out = new byte[compressor.maxCompressedLength(in.length)];
int compressedLength = compressor.compress(in, 0, in.length, out, 0);
getLogger().debug("Compressed %d bytes to %d", in.length, compressedLength);
return ByteBuffer.allocate(Ints.BYTES + compressedLength)
.putInt(in.length)
.put(out, 0, compressedLength)
.array();
}
@Override
protected byte[] decompress(byte[] in)
{
byte[] out = null;
if(in != null) {
LZ4Decompressor decompressor = lz4Factory.decompressor();
int size = ByteBuffer.wrap(in).getInt();
out = new byte[size];
decompressor.decompress(in, Ints.BYTES, out, 0, out.length);
}
return out == null ? null : out;
}
}

View File

@ -53,9 +53,10 @@ public class MemcachedCache implements Cache
public static MemcachedCache create(final MemcachedCacheConfig config)
{
try {
SerializingTranscoder transcoder = new SerializingTranscoder(config.getMaxObjectSize());
// disable compression
transcoder.setCompressionThreshold(Integer.MAX_VALUE);
LZ4Transcoder transcoder = new LZ4Transcoder(config.getMaxObjectSize());
// always use compression
transcoder.setCompressionThreshold(0);
return new MemcachedCache(
new MemcachedClient(

View File

@ -0,0 +1,100 @@
/*
* 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.curator.announcement.Announcer;
import com.metamx.druid.initialization.ZkPathsConfig;
import org.apache.curator.utils.ZKPaths;
/**
*/
public abstract class AbstractDataSegmentAnnouncer implements DataSegmentAnnouncer
{
private static final Logger log = new Logger(AbstractDataSegmentAnnouncer.class);
private final DruidServerMetadata server;
private final ZkPathsConfig config;
private final Announcer announcer;
private final ObjectMapper jsonMapper;
private final Object lock = new Object();
private volatile boolean started = false;
protected AbstractDataSegmentAnnouncer(
DruidServerMetadata server,
ZkPathsConfig config,
Announcer announcer,
ObjectMapper jsonMapper
)
{
this.server = server;
this.config = config;
this.announcer = announcer;
this.jsonMapper = jsonMapper;
}
@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 %s with config[%s]", getClass(), config);
announcer.unannounce(makeAnnouncementPath());
started = false;
}
}
private String makeAnnouncementPath()
{
return ZKPaths.makePath(config.getAnnouncementsPath(), server.getName());
}
}

View File

@ -0,0 +1,290 @@
/*
* 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.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.curator.announcement.Announcer;
import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig;
import org.apache.curator.utils.ZKPaths;
import org.joda.time.DateTime;
import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
/**
*/
public class BatchDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer
{
private static final Logger log = new Logger(BatchDataSegmentAnnouncer.class);
private final ZkDataSegmentAnnouncerConfig config;
private final Announcer announcer;
private final ObjectMapper jsonMapper;
private final String liveSegmentLocation;
private final Set<SegmentZNode> availableZNodes = Sets.newHashSet();
private final Map<DataSegment, SegmentZNode> segmentLookup = Maps.newHashMap();
public BatchDataSegmentAnnouncer(
DruidServerMetadata server,
ZkDataSegmentAnnouncerConfig config,
Announcer announcer,
ObjectMapper jsonMapper
)
{
super(server, config, announcer, jsonMapper);
this.config = config;
this.announcer = announcer;
this.jsonMapper = jsonMapper;
this.liveSegmentLocation = ZKPaths.makePath(config.getLiveSegmentsPath(), server.getName());
}
@Override
public void announceSegment(DataSegment segment) throws IOException
{
int newBytesLen = jsonMapper.writeValueAsBytes(segment).length;
if (newBytesLen > config.getMaxNumBytes()) {
throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxNumBytes());
}
// create new batch
if (availableZNodes.isEmpty()) {
SegmentZNode availableZNode = new SegmentZNode(makeServedSegmentPath(new DateTime().toString()));
availableZNode.addSegment(segment);
log.info("Announcing segment[%s] at path[%s]", segment.getIdentifier(), availableZNode.getPath());
announcer.announce(availableZNode.getPath(), availableZNode.getBytes());
segmentLookup.put(segment, availableZNode);
availableZNodes.add(availableZNode);
} else { // update existing batch
Iterator<SegmentZNode> iter = availableZNodes.iterator();
boolean done = false;
while (iter.hasNext() && !done) {
SegmentZNode availableZNode = iter.next();
if (availableZNode.getBytes().length + newBytesLen < config.getMaxNumBytes()) {
availableZNode.addSegment(segment);
log.info("Announcing segment[%s] at path[%s]", segment.getIdentifier(), availableZNode.getPath());
announcer.update(availableZNode.getPath(), availableZNode.getBytes());
segmentLookup.put(segment, availableZNode);
if (availableZNode.getCount() >= config.getSegmentsPerNode()) {
availableZNodes.remove(availableZNode);
}
done = true;
}
}
}
}
@Override
public void unannounceSegment(DataSegment segment) throws IOException
{
final SegmentZNode segmentZNode = segmentLookup.remove(segment);
segmentZNode.removeSegment(segment);
log.info("Unannouncing segment[%s] at path[%s]", segment.getIdentifier(), segmentZNode.getPath());
if (segmentZNode.getCount() == 0) {
availableZNodes.remove(segmentZNode);
announcer.unannounce(segmentZNode.getPath());
} else {
announcer.update(segmentZNode.getPath(), segmentZNode.getBytes());
availableZNodes.add(segmentZNode);
}
}
@Override
public void announceSegments(Iterable<DataSegment> segments) throws IOException
{
SegmentZNode segmentZNode = new SegmentZNode(makeServedSegmentPath(new DateTime().toString()));
Set<DataSegment> batch = Sets.newHashSet();
int byteSize = 0;
int count = 0;
for (DataSegment segment : segments) {
int newBytesLen = jsonMapper.writeValueAsBytes(segment).length;
if (newBytesLen > config.getMaxNumBytes()) {
throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxNumBytes());
}
if (count >= config.getSegmentsPerNode() || byteSize + newBytesLen > config.getMaxNumBytes()) {
segmentZNode.addSegments(batch);
announcer.announce(segmentZNode.getPath(), segmentZNode.getBytes());
segmentZNode = new SegmentZNode(makeServedSegmentPath(new DateTime().toString()));
batch = Sets.newHashSet();
count = 0;
byteSize = 0;
}
log.info("Announcing segment[%s] at path[%s]", segment.getIdentifier(), segmentZNode.getPath());
segmentLookup.put(segment, segmentZNode);
batch.add(segment);
count++;
byteSize += newBytesLen;
}
segmentZNode.addSegments(batch);
announcer.announce(segmentZNode.getPath(), segmentZNode.getBytes());
}
@Override
public void unannounceSegments(Iterable<DataSegment> segments) throws IOException
{
for (DataSegment segment : segments) {
unannounceSegment(segment);
}
}
private String makeServedSegmentPath(String zNode)
{
return ZKPaths.makePath(liveSegmentLocation, zNode);
}
private class SegmentZNode
{
private final String path;
private byte[] bytes = new byte[]{};
private int count = 0;
public SegmentZNode(String path)
{
this.path = path;
}
public String getPath()
{
return path;
}
public int getCount()
{
return count;
}
public byte[] getBytes()
{
return bytes;
}
public Set<DataSegment> getSegments()
{
if (bytes.length == 0) {
return Sets.newHashSet();
}
try {
return jsonMapper.readValue(
bytes, new TypeReference<Set<DataSegment>>()
{
}
);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
public void addSegment(DataSegment segment)
{
Set<DataSegment> zkSegments = getSegments();
zkSegments.add(segment);
try {
bytes = jsonMapper.writeValueAsBytes(zkSegments);
}
catch (Exception e) {
zkSegments.remove(segment);
throw Throwables.propagate(e);
}
count++;
}
public void addSegments(Set<DataSegment> segments)
{
Set<DataSegment> zkSegments = getSegments();
zkSegments.addAll(segments);
try {
bytes = jsonMapper.writeValueAsBytes(zkSegments);
}
catch (Exception e) {
zkSegments.removeAll(segments);
throw Throwables.propagate(e);
}
count += segments.size();
}
public void removeSegment(DataSegment segment)
{
Set<DataSegment> zkSegments = getSegments();
zkSegments.remove(segment);
try {
bytes = jsonMapper.writeValueAsBytes(zkSegments);
}
catch (Exception e) {
zkSegments.add(segment);
throw Throwables.propagate(e);
}
count--;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SegmentZNode that = (SegmentZNode) o;
if (!path.equals(that.path)) {
return false;
}
return true;
}
@Override
public int hashCode()
{
return path.hashCode();
}
}
}

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.coordination;
import com.metamx.druid.client.DataSegment;
@ -7,5 +26,10 @@ import java.io.IOException;
public interface DataSegmentAnnouncer
{
public void announceSegment(DataSegment segment) throws IOException;
public void unannounceSegment(DataSegment segment) throws IOException;
public void announceSegments(Iterable<DataSegment> segments) throws IOException;
public void unannounceSegments(Iterable<DataSegment> segments) throws IOException;
}

View File

@ -81,7 +81,7 @@ public class DruidServerMetadata
@Override
public String toString()
{
return "DruidServer{" +
return "DruidServerMetadata{" +
"name='" + name + '\'' +
", host='" + host + '\'' +
", maxSize=" + maxSize +

View File

@ -0,0 +1,92 @@
/*
* 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.databind.ObjectMapper;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.curator.announcement.Announcer;
import com.metamx.druid.initialization.ZkPathsConfig;
import java.io.IOException;
/**
* This class has the greatest name ever
*/
public class MultipleDataSegmentAnnouncerDataSegmentAnnouncer implements DataSegmentAnnouncer
{
private final Iterable<AbstractDataSegmentAnnouncer> dataSegmentAnnouncers;
public MultipleDataSegmentAnnouncerDataSegmentAnnouncer(
Iterable<AbstractDataSegmentAnnouncer> dataSegmentAnnouncers
)
{
this.dataSegmentAnnouncers = dataSegmentAnnouncers;
}
@LifecycleStart
public void start()
{
for (AbstractDataSegmentAnnouncer dataSegmentAnnouncer : dataSegmentAnnouncers) {
dataSegmentAnnouncer.start();
}
}
@LifecycleStop
public void stop()
{
for (AbstractDataSegmentAnnouncer dataSegmentAnnouncer : dataSegmentAnnouncers) {
dataSegmentAnnouncer.stop();
}
}
@Override
public void announceSegment(DataSegment segment) throws IOException
{
for (DataSegmentAnnouncer dataSegmentAnnouncer : dataSegmentAnnouncers) {
dataSegmentAnnouncer.announceSegment(segment);
}
}
@Override
public void unannounceSegment(DataSegment segment) throws IOException
{
for (DataSegmentAnnouncer dataSegmentAnnouncer : dataSegmentAnnouncers) {
dataSegmentAnnouncer.unannounceSegment(segment);
}
}
@Override
public void announceSegments(Iterable<DataSegment> segments) throws IOException
{
for (DataSegmentAnnouncer dataSegmentAnnouncer : dataSegmentAnnouncers) {
dataSegmentAnnouncer.announceSegments(segments);
}
}
@Override
public void unannounceSegments(Iterable<DataSegment> segments) throws IOException
{
for (DataSegmentAnnouncer dataSegmentAnnouncer : dataSegmentAnnouncers) {
dataSegmentAnnouncer.unannounceSegments(segments);
}
}
}

View File

@ -19,7 +19,6 @@
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.google.inject.Inject;
@ -33,14 +32,10 @@ import org.apache.curator.utils.ZKPaths;
import java.io.IOException;
public class CuratorDataSegmentAnnouncer implements DataSegmentAnnouncer
public class SingleDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer
{
private static final Logger log = new Logger(CuratorDataSegmentAnnouncer.class);
private static final Logger log = new Logger(SingleDataSegmentAnnouncer.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;
@ -49,55 +44,20 @@ public class CuratorDataSegmentAnnouncer implements DataSegmentAnnouncer
@Inject
public CuratorDataSegmentAnnouncer(
public SingleDataSegmentAnnouncer(
DruidServerMetadata server,
ZkPathsConfig config,
Announcer announcer,
ObjectMapper jsonMapper
)
{
this.server = server;
this.config = config;
super(server, config, announcer, jsonMapper);
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);
@ -112,8 +72,20 @@ public class CuratorDataSegmentAnnouncer implements DataSegmentAnnouncer
announcer.unannounce(path);
}
private String makeAnnouncementPath() {
return ZKPaths.makePath(config.getAnnouncementsPath(), server.getName());
@Override
public void announceSegments(Iterable<DataSegment> segments) throws IOException
{
for (DataSegment segment : segments) {
announceSegment(segment);
}
}
@Override
public void unannounceSegments(Iterable<DataSegment> segments) throws IOException
{
for (DataSegment segment : segments) {
unannounceSegment(segment);
}
}
private String makeServedSegmentPath(DataSegment segment)

View File

@ -32,4 +32,8 @@ public abstract class CuratorConfig
@Config("druid.zk.service.sessionTimeoutMs")
@Default("30000")
public abstract int getZkSessionTimeoutMs();
@Config("druid.curator.compress")
@Default("false")
public abstract boolean enableCompression();
}

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.curator.announcement;
import com.google.common.base.Throwables;
@ -6,6 +25,7 @@ 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.ISE;
import com.metamx.common.Pair;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
@ -21,11 +41,14 @@ 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 org.apache.zookeeper.data.Stat;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicReference;
@ -42,6 +65,7 @@ public class Announcer
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 final List<String> parentsIBuilt = new CopyOnWriteArrayList<String>();
private boolean started = false;
@ -92,6 +116,15 @@ public class Announcer
unannounce(ZKPaths.makePath(basePath, announcementPath));
}
}
for (String parent : parentsIBuilt) {
try {
curator.delete().forPath(parent);
}
catch (Exception e) {
log.info(e, "Unable to delete parent[%s], boooo.", parent);
}
}
}
}
@ -99,7 +132,7 @@ public class Announcer
* 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 path The path to announce at
* @param bytes The payload to announce
*/
public void announce(String path, byte[] bytes)
@ -114,10 +147,19 @@ public class Announcer
final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
final String parentPath = pathAndNode.getPath();
boolean buildParentPath = false;
ConcurrentMap<String, byte[]> subPaths = announcements.get(parentPath);
if (subPaths == null) {
try {
if (curator.checkExists().forPath(parentPath) == null) {
buildParentPath = true;
}
}
catch (Exception e) {
log.debug(e, "Problem checking if the parent existed, ignoring.");
}
// I don't have a watcher on this path yet, create a Map and start watching.
announcements.putIfAbsent(parentPath, new MapMaker().<String, byte[]>makeMap());
@ -127,7 +169,7 @@ public class Announcer
// Synchronize to make sure that I only create a listener once.
synchronized (finalSubPaths) {
if (! listeners.containsKey(parentPath)) {
if (!listeners.containsKey(parentPath)) {
final PathChildrenCache cache = factory.make(curator, parentPath);
cache.getListenable().addListener(
new PathChildrenCacheListener()
@ -186,17 +228,15 @@ public class Announcer
}
);
try {
synchronized (toAnnounce) {
if (started) {
cache.start();
listeners.put(parentPath, cache);
synchronized (toAnnounce) {
if (started) {
if (buildParentPath) {
createPath(parentPath);
}
startCache(cache);
listeners.put(parentPath, cache);
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
@ -208,11 +248,11 @@ public class Announcer
if (started) {
byte[] oldBytes = subPaths.putIfAbsent(pathAndNode.getNode(), bytes);
if (oldBytes != null) {
throw new IAE("Already announcing[%s], cannot announce it twice.", path);
if (oldBytes == null) {
created = true;
} else if (!Arrays.equals(oldBytes, bytes)) {
throw new IAE("Cannot reannounce different values under the same path");
}
created = true;
}
}
@ -226,15 +266,48 @@ public class Announcer
}
}
public void update(final String path, final byte[] bytes)
{
final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
final String parentPath = pathAndNode.getPath();
final String nodePath = pathAndNode.getNode();
ConcurrentMap<String, byte[]> subPaths = announcements.get(parentPath);
if (subPaths == null || subPaths.get(nodePath) == null) {
throw new ISE("Cannot update a path[%s] that hasn't been announced!", path);
}
synchronized (toAnnounce) {
try {
byte[] oldBytes = subPaths.get(nodePath);
if (!Arrays.equals(oldBytes, bytes)) {
subPaths.put(nodePath, bytes);
updateAnnouncement(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);
}
private Stat updateAnnouncement(final String path, final byte[] value) throws Exception
{
return curator.setData().compressed().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.
*
* <p/>
* If you need to completely clear all the state of what is being watched and announced, stop() the Announcer.
*
* @param path
@ -265,4 +338,26 @@ public class Announcer
throw Throwables.propagate(e);
}
}
private void startCache(PathChildrenCache cache)
{
try {
cache.start();
}
catch (Exception e) {
Closeables.closeQuietly(cache);
throw Throwables.propagate(e);
}
}
private void createPath(String parentPath)
{
try {
curator.create().creatingParentsIfNeeded().forPath(parentPath);
parentsIBuilt.add(parentPath);
}
catch (Exception e) {
log.info(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
}
}
}

View File

@ -135,6 +135,11 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>
}
}
public InventoryManagerConfig getConfig()
{
return config;
}
public ContainerClass getInventoryValue(String containerKey)
{
final ContainerHolder containerHolder = containers.get(containerKey);
@ -290,11 +295,18 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>
switch (event.getType()) {
case CHILD_ADDED:
case CHILD_UPDATED:
final InventoryClass inventory = strategy.deserializeInventory(child.getData());
final InventoryClass addedInventory = strategy.deserializeInventory(child.getData());
synchronized (holder) {
holder.setContainer(strategy.addInventory(holder.getContainer(), inventoryKey, inventory));
holder.setContainer(strategy.addInventory(holder.getContainer(), inventoryKey, addedInventory));
}
break;
case CHILD_UPDATED:
final InventoryClass updatedInventory = strategy.deserializeInventory(child.getData());
synchronized (holder) {
holder.setContainer(strategy.updateInventory(holder.getContainer(), inventoryKey, updatedInventory));
}
break;

View File

@ -33,5 +33,6 @@ public interface CuratorInventoryManagerStrategy<ContainerClass, InventoryClass>
public void deadContainer(ContainerClass deadContainer);
public ContainerClass updateContainer(ContainerClass oldContainer, ContainerClass newContainer);
public ContainerClass addInventory(ContainerClass container, String inventoryKey, InventoryClass inventory);
public ContainerClass updateInventory(ContainerClass container, String inventoryKey, InventoryClass inventory);
public ContainerClass removeInventory(ContainerClass container, String inventoryKey);
}

View File

@ -86,13 +86,13 @@ public class Initialization
/**
* Load properties.
* Properties are layered:
*
* <p/>
* # stored in zookeeper
* # runtime.properties file,
* # cmdLine -D
*
* <p/>
* command line overrides runtime.properties which overrides zookeeper
*
* <p/>
* Idempotent. Thread-safe. Properties are only loaded once.
* If property druid.zk.service.host is not set then do not load properties from zookeeper.
*
@ -210,10 +210,9 @@ public class Initialization
CuratorFrameworkFactory.builder()
.connectString(curatorConfig.getZkHosts())
.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();
.retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30))
.compressionProvider(new PotentiallyGzippedCompressionProvider(curatorConfig.enableCompression()))
.build();
lifecycle.addHandler(
new Lifecycle.Handler()
@ -346,9 +345,9 @@ public class Initialization
}
public static RequestLogger makeFileRequestLogger(
ObjectMapper objectMapper,
ScheduledExecutorFactory factory,
Properties props
ObjectMapper objectMapper,
ScheduledExecutorFactory factory,
Properties props
) throws IOException
{
return new FileRequestLogger(

View File

@ -0,0 +1,21 @@
package com.metamx.druid.initialization;
import org.skife.config.Config;
import org.skife.config.Default;
/**
*/
public abstract class ZkDataSegmentAnnouncerConfig extends ZkPathsConfig
{
@Config("druid.zk.segmentsPerNode")
@Default("50")
public abstract int getSegmentsPerNode();
@Config("druid.zk.maxNumBytesPerNode")
@Default("512000")
public abstract long getMaxNumBytes();
@Config("druid.announcer.type")
@Default("legacy")
public abstract String getAnnouncerType();
}

View File

@ -21,6 +21,7 @@ package com.metamx.druid.initialization;
import org.apache.curator.utils.ZKPaths;
import org.skife.config.Config;
import org.skife.config.Default;
public abstract class ZkPathsConfig
{
@ -45,6 +46,12 @@ public abstract class ZkPathsConfig
return defaultPath("servedSegments");
}
@Config("druid.zk.paths.liveSegmentsPath")
public String getLiveSegmentsPath()
{
return defaultPath("segments");
}
@Config("druid.zk.paths.loadQueuePath")
public String getLoadQueuePath()
{

View File

@ -20,6 +20,7 @@
package com.metamx.druid.query;
import com.google.common.base.Function;
import com.google.common.base.Predicates;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
@ -34,7 +35,6 @@ import com.metamx.druid.Query;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
@ -77,12 +77,14 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
{
this.exec = exec;
this.ordering = ordering;
this.queryables = Iterables.unmodifiableIterable(queryables);
this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull()));
}
@Override
public Sequence<T> run(final Query<T> query)
{
final int priority = Integer.parseInt(query.getContextValue("priority", "0"));
return new BaseSequence<T, Iterator<T>>(
new BaseSequence.IteratorMaker<T, Iterator<T>>()
{
@ -99,7 +101,7 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
public Future<List<T>> apply(final QueryRunner<T> input)
{
return exec.submit(
new Callable<List<T>>()
new PrioritizedCallable<List<T>>(priority)
{
@Override
public List<T> call() throws Exception

View File

@ -0,0 +1,127 @@
/*
* 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.query;
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;
/**
*/
public class DelegatingExecutorService implements ExecutorService
{
private final ExecutorService delegate;
public DelegatingExecutorService(ExecutorService delegate)
{
this.delegate = delegate;
}
@Override
public void shutdown()
{
delegate.shutdown();
}
@Override
public List<Runnable> shutdownNow()
{
return delegate.shutdownNow();
}
@Override
public boolean isShutdown()
{
return delegate.isShutdown();
}
@Override
public boolean isTerminated()
{
return delegate.isTerminated();
}
@Override
public boolean awaitTermination(long l, TimeUnit timeUnit) throws InterruptedException
{
return delegate.awaitTermination(l, timeUnit);
}
@Override
public <T> Future<T> submit(Callable<T> tCallable)
{
return delegate.submit(tCallable);
}
@Override
public <T> Future<T> submit(Runnable runnable, T t)
{
return delegate.submit(runnable, t);
}
@Override
public Future<?> submit(Runnable runnable)
{
return delegate.submit(runnable);
}
@Override
public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> callables) throws InterruptedException
{
return delegate.invokeAll(callables);
}
@Override
public <T> List<Future<T>> invokeAll(
Collection<? extends Callable<T>> callables,
long l,
TimeUnit timeUnit
) throws InterruptedException
{
return delegate.invokeAll(callables, l, timeUnit);
}
@Override
public <T> T invokeAny(Collection<? extends Callable<T>> callables) throws InterruptedException, ExecutionException
{
return delegate.invokeAny(callables);
}
@Override
public <T> T invokeAny(
Collection<? extends Callable<T>> callables,
long l,
TimeUnit timeUnit
) throws InterruptedException, ExecutionException, TimeoutException
{
return delegate.invokeAny(callables, l, timeUnit);
}
@Override
public void execute(Runnable runnable)
{
delegate.execute(runnable);
}
}

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.query;
import com.metamx.common.lifecycle.LifecycleStop;
@ -5,14 +24,19 @@ import com.metamx.druid.guice.ManageLifecycle;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.AbstractExecutorService;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.RunnableFuture;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
@ManageLifecycle
public class MetricsEmittingExecutorService extends AbstractExecutorService
public class MetricsEmittingExecutorService extends DelegatingExecutorService
{
private final ExecutorService base;
private final ServiceEmitter emitter;
@ -24,42 +48,13 @@ public class MetricsEmittingExecutorService extends AbstractExecutorService
ServiceMetricEvent.Builder metricBuilder
)
{
super(base);
this.base = base;
this.emitter = emitter;
this.metricBuilder = metricBuilder;
}
@Override
public void shutdown()
{
base.shutdown();
}
@Override
@LifecycleStop
public List<Runnable> shutdownNow()
{
return base.shutdownNow();
}
@Override
public boolean isShutdown()
{
return base.isShutdown();
}
@Override
public boolean isTerminated()
{
return base.isTerminated();
}
@Override
public boolean awaitTermination(long l, TimeUnit timeUnit) throws InterruptedException
{
return base.awaitTermination(l, timeUnit);
}
@Override
public void execute(Runnable runnable)
{

View File

@ -0,0 +1,39 @@
/*
* 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.query;
import java.util.concurrent.Callable;
/**
*/
public abstract class PrioritizedCallable<T> implements Callable<T>
{
final int priority;
public PrioritizedCallable(int priority)
{
this.priority = priority;
}
public int getPriority()
{
return priority;
}
}

View File

@ -0,0 +1,158 @@
/*
* 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.query;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.concurrent.ExecutorServiceConfig;
import com.metamx.common.lifecycle.Lifecycle;
import java.util.List;
import java.util.concurrent.AbstractExecutorService;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.FutureTask;
import java.util.concurrent.PriorityBlockingQueue;
import java.util.concurrent.RunnableFuture;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
/**
*/
public class PrioritizedExecutorService extends AbstractExecutorService
{
public static PrioritizedExecutorService create(Lifecycle lifecycle, ExecutorServiceConfig config)
{
final PrioritizedExecutorService service = new PrioritizedExecutorService(
new ThreadPoolExecutor(
config.getNumThreads(),
config.getNumThreads(),
0L,
TimeUnit.MILLISECONDS,
new PriorityBlockingQueue<Runnable>(),
new ThreadFactoryBuilder().setDaemon(true).setNameFormat(config.getFormatString()).build()
)
);
lifecycle.addHandler(
new Lifecycle.Handler()
{
@Override
public void start() throws Exception
{
}
@Override
public void stop()
{
service.shutdownNow();
}
}
);
return service;
}
private static final int DEFAULT_PRIORITY = 0;
private final ThreadPoolExecutor threadPoolExecutor;
public PrioritizedExecutorService(
ThreadPoolExecutor threadPoolExecutor
)
{
this.threadPoolExecutor = threadPoolExecutor;
}
@Override
public void shutdown()
{
threadPoolExecutor.shutdown();
}
@Override
public List<Runnable> shutdownNow()
{
return threadPoolExecutor.shutdownNow();
}
@Override
public boolean isShutdown()
{
return threadPoolExecutor.isShutdown();
}
@Override
public boolean isTerminated()
{
return threadPoolExecutor.isTerminated();
}
@Override
public boolean awaitTermination(long l, TimeUnit timeUnit) throws InterruptedException
{
return threadPoolExecutor.awaitTermination(l, timeUnit);
}
@Override
public void execute(Runnable runnable)
{
threadPoolExecutor.execute(runnable);
}
@Override
protected <T> RunnableFuture<T> newTaskFor(final Callable<T> tCallable)
{
Callable<T> theCallable = tCallable;
if (!(tCallable instanceof PrioritizedCallable)) {
theCallable = new PrioritizedCallable<T>(DEFAULT_PRIORITY)
{
@Override
public T call() throws Exception
{
return tCallable.call();
}
};
}
return new PrioritizedFuture<T>((PrioritizedCallable) theCallable);
}
private static class PrioritizedFuture<V> extends FutureTask<V> implements Comparable<PrioritizedFuture>
{
private final PrioritizedCallable<V> callable;
public PrioritizedFuture(PrioritizedCallable<V> callable)
{
super(callable);
this.callable = callable;
}
public int getPriority()
{
return callable.getPriority();
}
@Override
public int compareTo(PrioritizedFuture future)
{
return -Ints.compare(getPriority(), future.getPriority());
}
}
}

View File

@ -46,13 +46,16 @@ import org.joda.time.Minutes;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Set;
public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAnalysis, SegmentMetadataQuery>
{
private static final TypeReference<SegmentAnalysis> TYPE_REFERENCE = new TypeReference<SegmentAnalysis>(){};
private static final TypeReference<SegmentAnalysis> TYPE_REFERENCE = new TypeReference<SegmentAnalysis>()
{
};
private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4};
@Override
@ -228,6 +231,6 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
{
return left.getId().compareTo(right.getId());
}
};
}.nullsFirst();
}
}

View File

@ -22,6 +22,8 @@ package com.metamx.druid.query.timeboundary;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Function;
import com.google.common.base.Functions;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.metamx.common.guava.MergeSequence;
@ -59,13 +61,29 @@ public class TimeBoundaryQueryQueryToolChest
};
@Override
public <T extends LogicalSegment> List<T> filterSegments(TimeBoundaryQuery query, List<T> input)
public <T extends LogicalSegment> List<T> filterSegments(TimeBoundaryQuery query, List<T> segments)
{
if(input.size() <= 1) {
return input;
if (segments.size() <= 1) {
return segments;
}
return Lists.newArrayList(input.get(0), input.get(input.size() - 1));
final T first = segments.get(0);
final T second = segments.get(segments.size() - 1);
return Lists.newArrayList(
Iterables.filter(
segments,
new Predicate<T>()
{
@Override
public boolean apply(T input)
{
return input.getInterval().overlaps(first.getInterval()) || input.getInterval()
.overlaps(second.getInterval());
}
}
)
);
}
@Override

View File

@ -0,0 +1,231 @@
/*
* 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.base.Stopwatch;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.MoreExecutors;
import com.metamx.common.ISE;
import com.metamx.druid.coordination.BatchDataSegmentAnnouncer;
import com.metamx.druid.coordination.DruidServerMetadata;
import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider;
import com.metamx.druid.curator.announcement.Announcer;
import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig;
import com.metamx.druid.initialization.ZkPathsConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import junit.framework.Assert;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.ExponentialBackoffRetry;
import org.apache.curator.test.TestingCluster;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import java.util.Set;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
/**
*/
public class BatchServerInventoryViewTest
{
private static final String testBasePath = "/test";
private static final Joiner joiner = Joiner.on("/");
private TestingCluster testingCluster;
private CuratorFramework cf;
private ObjectMapper jsonMapper;
private Announcer announcer;
private BatchDataSegmentAnnouncer segmentAnnouncer;
private Set<DataSegment> testSegments;
private BatchServerInventoryView batchServerInventoryView;
@Before
public void setUp() throws Exception
{
testingCluster = new TestingCluster(1);
testingCluster.start();
cf = CuratorFrameworkFactory.builder()
.connectString(testingCluster.getConnectString())
.retryPolicy(new ExponentialBackoffRetry(1, 10))
.compressionProvider(new PotentiallyGzippedCompressionProvider(true))
.build();
cf.start();
cf.create().creatingParentsIfNeeded().forPath(testBasePath);
jsonMapper = new DefaultObjectMapper();
announcer = new Announcer(
cf,
MoreExecutors.sameThreadExecutor()
);
announcer.start();
segmentAnnouncer = new BatchDataSegmentAnnouncer(
new DruidServerMetadata(
"id",
"host",
Long.MAX_VALUE,
"type",
"tier"
),
new ZkDataSegmentAnnouncerConfig()
{
@Override
public String getZkBasePath()
{
return testBasePath;
}
@Override
public int getSegmentsPerNode()
{
return 50;
}
@Override
public long getMaxNumBytes()
{
return 100000;
}
@Override
public String getAnnouncerType()
{
return "batch";
}
},
announcer,
jsonMapper
);
segmentAnnouncer.start();
testSegments = Sets.newHashSet();
for (int i = 0; i < 100; i++) {
testSegments.add(makeSegment(i));
}
batchServerInventoryView = new BatchServerInventoryView(
new ServerInventoryViewConfig()
{
@Override
public int getRemovedSegmentLifetime()
{
return 0;
}
@Override
public String getAnnouncerType()
{
return "batch";
}
},
new ZkPathsConfig()
{
@Override
public String getZkBasePath()
{
return testBasePath;
}
},
cf,
Executors.newSingleThreadExecutor(),
jsonMapper
);
batchServerInventoryView.start();
}
@After
public void tearDown() throws Exception
{
batchServerInventoryView.stop();
segmentAnnouncer.stop();
announcer.stop();
cf.close();
testingCluster.stop();
}
@Test
public void testRun() throws Exception
{
segmentAnnouncer.announceSegments(testSegments);
waitForSync();
DruidServer server = Iterables.get(batchServerInventoryView.getInventory(), 0);
Set<DataSegment> segments = Sets.newHashSet(server.getSegments().values());
Assert.assertEquals(testSegments, segments);
DataSegment segment1 = makeSegment(101);
DataSegment segment2 = makeSegment(102);
segmentAnnouncer.announceSegment(segment1);
segmentAnnouncer.announceSegment(segment2);
testSegments.add(segment1);
testSegments.add(segment2);
waitForSync();
Assert.assertEquals(testSegments, Sets.newHashSet(server.getSegments().values()));
segmentAnnouncer.unannounceSegment(segment1);
segmentAnnouncer.unannounceSegment(segment2);
testSegments.remove(segment1);
testSegments.remove(segment2);
waitForSync();
Assert.assertEquals(testSegments, Sets.newHashSet(server.getSegments().values()));
}
private DataSegment makeSegment(int offset)
{
return DataSegment.builder()
.dataSource("foo")
.interval(
new Interval(
new DateTime("2013-01-01").plusDays(offset),
new DateTime("2013-01-02").plusDays(offset)
)
)
.version(new DateTime().toString())
.build();
}
private void waitForSync() throws Exception
{
Stopwatch stopwatch = new Stopwatch().start();
while (Iterables.get(batchServerInventoryView.getInventory(), 0).getSegments().size() != testSegments.size()) {
Thread.sleep(500);
if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 5000) {
throw new ISE("BatchServerInventoryView is not updating");
}
}
}
}

View File

@ -52,8 +52,8 @@ import java.util.concurrent.TimeoutException;
*/
public class MemcachedCacheTest
{
private static final byte[] HI = "hi".getBytes();
private static final byte[] HO = "ho".getBytes();
private static final byte[] HI = "hiiiiiiiiiiiiiiiiiii".getBytes();
private static final byte[] HO = "hooooooooooooooooooo".getBytes();
private MemcachedCache cache;
@Before
@ -124,7 +124,13 @@ public class MemcachedCacheTest
class MockMemcachedClient implements MemcachedClientIF
{
private final ConcurrentMap<String, CachedData> theMap = new ConcurrentHashMap<String, CachedData>();
private final Transcoder<Object> transcoder = new SerializingTranscoder();
private final SerializingTranscoder transcoder;
public MockMemcachedClient()
{
transcoder = new LZ4Transcoder();
transcoder.setCompressionThreshold(0);
}
@Override
public Collection<SocketAddress> getAvailableServers()

View File

@ -0,0 +1,247 @@
/*
* 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.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Joiner;
import com.google.common.base.Throwables;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.MoreExecutors;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider;
import com.metamx.druid.curator.announcement.Announcer;
import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import junit.framework.Assert;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.ExponentialBackoffRetry;
import org.apache.curator.test.TestingCluster;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
/**
*/
public class BatchDataSegmentAnnouncerTest
{
private static final String testBasePath = "/test";
private static final String testSegmentsPath = "/test/segments/id";
private static final Joiner joiner = Joiner.on("/");
private TestingCluster testingCluster;
private CuratorFramework cf;
private ObjectMapper jsonMapper;
private Announcer announcer;
private SegmentReader segmentReader;
private BatchDataSegmentAnnouncer segmentAnnouncer;
private Set<DataSegment> testSegments;
@Before
public void setUp() throws Exception
{
testingCluster = new TestingCluster(1);
testingCluster.start();
cf = CuratorFrameworkFactory.builder()
.connectString(testingCluster.getConnectString())
.retryPolicy(new ExponentialBackoffRetry(1, 10))
.compressionProvider(new PotentiallyGzippedCompressionProvider(false))
.build();
cf.start();
cf.create().creatingParentsIfNeeded().forPath(testBasePath);
jsonMapper = new DefaultObjectMapper();
announcer = new Announcer(
cf,
MoreExecutors.sameThreadExecutor()
);
announcer.start();
segmentReader = new SegmentReader(cf, jsonMapper);
segmentAnnouncer = new BatchDataSegmentAnnouncer(
new DruidServerMetadata(
"id",
"host",
Long.MAX_VALUE,
"type",
"tier"
),
new ZkDataSegmentAnnouncerConfig()
{
@Override
public String getZkBasePath()
{
return testBasePath;
}
@Override
public int getSegmentsPerNode()
{
return 50;
}
@Override
public long getMaxNumBytes()
{
return 100000;
}
@Override
public String getAnnouncerType()
{
return "batch";
}
},
announcer,
jsonMapper
);
segmentAnnouncer.start();
testSegments = Sets.newHashSet();
for (int i = 0; i < 100; i++) {
testSegments.add(makeSegment(i));
}
}
@After
public void tearDown() throws Exception
{
segmentAnnouncer.stop();
announcer.stop();
cf.close();
testingCluster.stop();
}
@Test
public void testSingleAnnounce() throws Exception
{
Iterator<DataSegment> segIter = testSegments.iterator();
DataSegment firstSegment = segIter.next();
DataSegment secondSegment = segIter.next();
segmentAnnouncer.announceSegment(firstSegment);
List<String> zNodes = cf.getChildren().forPath(testSegmentsPath);
for (String zNode : zNodes) {
Set<DataSegment> segments = segmentReader.read(joiner.join(testSegmentsPath, zNode));
Assert.assertEquals(segments.iterator().next(), firstSegment);
}
segmentAnnouncer.announceSegment(secondSegment);
for (String zNode : zNodes) {
Set<DataSegment> segments = segmentReader.read(joiner.join(testSegmentsPath, zNode));
Assert.assertEquals(Sets.newHashSet(firstSegment, secondSegment), segments);
}
segmentAnnouncer.unannounceSegment(firstSegment);
for (String zNode : zNodes) {
Set<DataSegment> segments = segmentReader.read(joiner.join(testSegmentsPath, zNode));
Assert.assertEquals(segments.iterator().next(), secondSegment);
}
segmentAnnouncer.unannounceSegment(secondSegment);
Assert.assertTrue(cf.getChildren().forPath(testSegmentsPath).isEmpty());
}
@Test
public void testBatchAnnounce() throws Exception
{
segmentAnnouncer.announceSegments(testSegments);
List<String> zNodes = cf.getChildren().forPath(testSegmentsPath);
Assert.assertTrue(zNodes.size() == 2);
Set<DataSegment> allSegments = Sets.newHashSet();
for (String zNode : zNodes) {
allSegments.addAll(segmentReader.read(joiner.join(testSegmentsPath, zNode)));
}
Assert.assertEquals(allSegments, testSegments);
segmentAnnouncer.unannounceSegments(testSegments);
Assert.assertTrue(cf.getChildren().forPath(testSegmentsPath).isEmpty());
}
@Test
public void testMultipleBatchAnnounce() throws Exception
{
for (int i = 0; i < 10; i++) {
testBatchAnnounce();
}
}
private DataSegment makeSegment(int offset)
{
return DataSegment.builder()
.dataSource("foo")
.interval(
new Interval(
new DateTime("2013-01-01").plusDays(offset),
new DateTime("2013-01-02").plusDays(offset)
)
)
.version(new DateTime().toString())
.build();
}
private class SegmentReader
{
private final CuratorFramework cf;
private final ObjectMapper jsonMapper;
public SegmentReader(CuratorFramework cf, ObjectMapper jsonMapper)
{
this.cf = cf;
this.jsonMapper = jsonMapper;
}
public Set<DataSegment> read(String path)
{
try {
if (cf.checkExists().forPath(path) != null) {
return jsonMapper.readValue(
cf.getData().forPath(path), new TypeReference<Set<DataSegment>>()
{
}
);
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
return Sets.newHashSet();
}
}
}

View File

@ -27,6 +27,8 @@ 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.apache.curator.utils.ZKPaths;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -60,7 +62,6 @@ public class AnnouncerTest extends CuratorTestBase
public void testSanity() throws Exception
{
curator.start();
curator.create().forPath("/somewhere");
Announcer announcer = new Announcer(curator, exec);
final byte[] billy = "billy".getBytes();
@ -161,4 +162,54 @@ public class AnnouncerTest extends CuratorTestBase
announcer.stop();
}
}
@Test
public void testCleansUpItsLittleTurdlings() throws Exception
{
curator.start();
Announcer announcer = new Announcer(curator, exec);
final byte[] billy = "billy".getBytes();
final String testPath = "/somewhere/test2";
final String parent = ZKPaths.getPathAndNode(testPath).getPath();
announcer.start();
Assert.assertNull(curator.checkExists().forPath(parent));
announcer.announce(testPath, billy);
Assert.assertNotNull(curator.checkExists().forPath(parent));
announcer.stop();
Assert.assertNull(curator.checkExists().forPath(parent));
}
@Test
public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception
{
curator.start();
Announcer announcer = new Announcer(curator, exec);
final byte[] billy = "billy".getBytes();
final String testPath = "/somewhere/test2";
final String parent = ZKPaths.getPathAndNode(testPath).getPath();
curator.create().forPath(parent);
final Stat initialStat = curator.checkExists().forPath(parent);
announcer.start();
Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
announcer.announce(testPath, billy);
Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
announcer.stop();
Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
}
}

View File

@ -43,14 +43,11 @@ public class CuratorInventoryManagerTest extends CuratorTestBase
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[]{});
curator.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath("/container/billy", new byte[]{});
Assert.assertTrue(timing.awaitLatch(containerLatch));
strategy.setNewContainerLatch(null);
@ -60,7 +57,7 @@ public class CuratorInventoryManagerTest extends CuratorTestBase
CountDownLatch inventoryLatch = new CountDownLatch(2);
strategy.setNewInventoryLatch(inventoryLatch);
curator.create().withMode(CreateMode.EPHEMERAL).forPath("/inventory/billy/1", Ints.toByteArray(100));
curator.create().creatingParentsIfNeeded().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));
@ -212,6 +209,14 @@ public class CuratorInventoryManagerTest extends CuratorTestBase
return container;
}
@Override
public Map<String, Integer> updateInventory(
Map<String, Integer> container, String inventoryKey, Integer inventory
)
{
return addInventory(container, inventoryKey, inventory);
}
@Override
public Map<String, Integer> removeInventory(Map<String, Integer> container, String inventoryKey)
{

View File

@ -0,0 +1,135 @@
/*
* 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.query;
import com.google.common.collect.ImmutableList;
import com.metamx.common.concurrent.ExecutorServiceConfig;
import com.metamx.common.lifecycle.Lifecycle;
import junit.framework.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.List;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
/**
*/
public class PrioritizedExecutorServiceTest
{
private ExecutorService exec;
private CountDownLatch latch;
private CountDownLatch finishLatch;
@Before
public void setUp() throws Exception
{
exec = PrioritizedExecutorService.create(
new Lifecycle(),
new ExecutorServiceConfig()
{
@Override
public String getFormatString()
{
return "test";
}
@Override
public int getNumThreads()
{
return 1;
}
}
);
latch = new CountDownLatch(1);
finishLatch = new CountDownLatch(3);
}
/**
* Submits a normal priority task to block the queue, followed by low, high, normal priority tasks.
* Tests to see that the high priority task is executed first, followed by the normal and low priority tasks.
*
* @throws Exception
*/
@Test
public void testSubmit() throws Exception
{
final ConcurrentLinkedQueue<Integer> order = new ConcurrentLinkedQueue<Integer>();
exec.submit(
new PrioritizedCallable<Void>(0)
{
@Override
public Void call() throws Exception
{
latch.await();
return null;
}
}
);
exec.submit(
new PrioritizedCallable<Void>(-1)
{
@Override
public Void call() throws Exception
{
order.add(-1);
finishLatch.countDown();
return null;
}
}
);
exec.submit(
new PrioritizedCallable<Void>(0)
{
@Override
public Void call() throws Exception
{
order.add(0);
finishLatch.countDown();
return null;
}
}
);
exec.submit(
new PrioritizedCallable<Void>(2)
{
@Override
public Void call() throws Exception
{
order.add(2);
finishLatch.countDown();
return null;
}
}
);
latch.countDown();
finishLatch.await();
Assert.assertTrue(order.size() == 3);
List<Integer> expected = ImmutableList.of(2, 0, -1);
Assert.assertEquals(expected, ImmutableList.copyOf(order));
}
}

View File

@ -0,0 +1,100 @@
/*
* 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.query.timeboundary;
import com.metamx.druid.LogicalSegment;
import junit.framework.Assert;
import org.joda.time.Interval;
import org.junit.Test;
import java.util.Arrays;
import java.util.List;
/**
*/
public class TimeBoundaryQueryQueryToolChestTest
{
@Test
public void testFilterSegments() throws Exception
{
List<LogicalSegment> segments = new TimeBoundaryQueryQueryToolChest().filterSegments(
null,
Arrays.asList(
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return new Interval("2013-01-01/P1D");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return new Interval("2013-01-01T01/PT1H");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return new Interval("2013-01-01T02/PT1H");
}
}
)
);
Assert.assertEquals(segments.size(), 3);
List<LogicalSegment> expected = Arrays.asList(
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return new Interval("2013-01-01/P1D");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return new Interval("2013-01-01T01/PT1H");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return new Interval("2013-01-01T02/PT1H");
}
}
);
for (int i = 0; i < segments.size(); i++) {
Assert.assertEquals(segments.get(i).getInterval(), expected.get(i).getInterval());
}
}
}

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.5.0-SNAPSHOT</version>
<version>0.5.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -21,6 +21,7 @@ package com.metamx.druid.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.primitives.Longs;
import com.metamx.druid.processing.ColumnSelectorFactory;
@ -40,6 +41,8 @@ public class CountAggregatorFactory implements AggregatorFactory
@JsonProperty("name") String name
)
{
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
this.name = name;
}

View File

@ -19,6 +19,7 @@
package com.metamx.druid.aggregation;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Doubles;
import com.metamx.druid.processing.FloatMetricSelector;
@ -28,14 +29,14 @@ import java.util.Comparator;
*/
public class DoubleSumAggregator implements Aggregator
{
static final Comparator COMPARATOR = new Comparator()
static final Comparator COMPARATOR = new Ordering()
{
@Override
public int compare(Object o, Object o1)
{
return Doubles.compare(((Number) o).doubleValue(), ((Number) o1).doubleValue());
}
};
}.nullsFirst();
static double combineValues(Object lhs, Object rhs)
{

View File

@ -21,6 +21,7 @@ package com.metamx.druid.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Doubles;
import com.metamx.druid.processing.ColumnSelectorFactory;
@ -44,6 +45,9 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory
@JsonProperty("fieldName") final String fieldName
)
{
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
this.name = name;
this.fieldName = fieldName;
}

View File

@ -22,6 +22,8 @@ package com.metamx.druid.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.primitives.Floats;
import com.google.common.primitives.Longs;
import com.metamx.druid.processing.ColumnSelectorFactory;
@ -49,12 +51,18 @@ public class HistogramAggregatorFactory implements AggregatorFactory
@JsonProperty("breaks") final List<Float> breaksList
)
{
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
this.name = name;
this.fieldName = fieldName;
this.breaksList = breaksList;
this.breaks = new float[breaksList.size()];
for(int i = 0; i < breaksList.size(); ++i) this.breaks[i] = breaksList.get(i);
this.breaksList = (breaksList == null) ? Lists.<Float>newArrayList() :breaksList;
this.breaks = new float[this.breaksList.size()];
for (int i = 0; i < this.breaksList.size(); ++i) {
this.breaks[i] = this.breaksList.get(i);
}
}
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
@ -95,14 +103,12 @@ public class HistogramAggregatorFactory implements AggregatorFactory
@Override
public Object deserialize(Object object)
{
if (object instanceof byte []) {
return Histogram.fromBytes((byte []) object);
}
else if (object instanceof ByteBuffer) {
if (object instanceof byte[]) {
return Histogram.fromBytes((byte[]) object);
} else if (object instanceof ByteBuffer) {
return Histogram.fromBytes((ByteBuffer) object);
}
else if(object instanceof String) {
byte[] bytes = Base64.decodeBase64(((String)object).getBytes(Charsets.UTF_8));
} else if (object instanceof String) {
byte[] bytes = Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8));
return Histogram.fromBytes(bytes);
}
return object;
@ -111,7 +117,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory
@Override
public Object finalizeComputation(Object object)
{
return ((Histogram)object).asVisual();
return ((Histogram) object).asVisual();
}
@Override
@ -149,7 +155,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory
@Override
public String getTypeName()
{
throw new UnsupportedOperationException("HistogramAggregatorFactory does not support getTypeName()");
throw new UnsupportedOperationException("HistogramAggregatorFactory does not support getTypeName()");
}
@Override

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.primitives.Doubles;
import com.metamx.druid.processing.ColumnSelectorFactory;
@ -51,7 +52,6 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
private final String fnCombine;
private final JavaScriptAggregator.ScriptAggregator compiledScript;
@JsonCreator
@ -63,6 +63,12 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
@JsonProperty("fnCombine") final String fnCombine
)
{
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
Preconditions.checkNotNull(fieldNames, "Must have a valid, non-null fieldNames");
Preconditions.checkNotNull(fnAggregate, "Must have a valid, non-null fnAggregate");
Preconditions.checkNotNull(fnReset, "Must have a valid, non-null fnReset");
Preconditions.checkNotNull(fnCombine, "Must have a valid, non-null fnCombine");
this.name = name;
this.fieldNames = fieldNames;
@ -83,7 +89,8 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
new com.google.common.base.Function<String, ObjectColumnSelector>()
{
@Override
public ObjectColumnSelector apply(@Nullable String s) {
public ObjectColumnSelector apply(@Nullable String s)
{
return columnFactory.makeObjectColumnSelector(s);
}
}
@ -101,7 +108,8 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
new com.google.common.base.Function<String, ObjectColumnSelector>()
{
@Override
public ObjectColumnSelector apply(@Nullable String s) {
public ObjectColumnSelector apply(@Nullable String s)
{
return columnSelectorFactory.makeObjectColumnSelector(s);
}
}
@ -148,7 +156,8 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
}
@JsonProperty
public List<String> getFieldNames() {
public List<String> getFieldNames()
{
return fieldNames;
}
@ -182,7 +191,7 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
try {
MessageDigest md = MessageDigest.getInstance("SHA-1");
byte[] fieldNameBytes = Joiner.on(",").join(fieldNames).getBytes(Charsets.UTF_8);
byte[] sha1 = md.digest((fnAggregate+fnReset+fnCombine).getBytes(Charsets.UTF_8));
byte[] sha1 = md.digest((fnAggregate + fnReset + fnCombine).getBytes(Charsets.UTF_8));
return ByteBuffer.allocate(1 + fieldNameBytes.length + sha1.length)
.put(CACHE_TYPE_ID)
@ -225,7 +234,11 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
'}';
}
public static JavaScriptAggregator.ScriptAggregator compileScript(final String aggregate, final String reset, final String combine)
public static JavaScriptAggregator.ScriptAggregator compileScript(
final String aggregate,
final String reset,
final String combine
)
{
final ContextFactory contextFactory = ContextFactory.getGlobal();
Context context = contextFactory.enterContext();
@ -234,8 +247,8 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
final ScriptableObject scope = context.initStandardObjects();
final Function fnAggregate = context.compileFunction(scope, aggregate, "aggregate", 1, null);
final Function fnReset = context.compileFunction(scope, reset, "reset", 1, null);
final Function fnCombine = context.compileFunction(scope, combine, "combine", 1, null);
final Function fnReset = context.compileFunction(scope, reset, "reset", 1, null);
final Function fnCombine = context.compileFunction(scope, combine, "combine", 1, null);
Context.exit();
return new JavaScriptAggregator.ScriptAggregator()
@ -244,7 +257,9 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
public double aggregate(final double current, final ObjectColumnSelector[] selectorList)
{
Context cx = Context.getCurrentContext();
if(cx == null) cx = contextFactory.enterContext();
if (cx == null) {
cx = contextFactory.enterContext();
}
final int size = selectorList.length;
final Object[] args = new Object[size + 1];
@ -292,8 +307,9 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
}
@Override
public void close() {
if(Context.getCurrentContext() != null) {
public void close()
{
if (Context.getCurrentContext() != null) {
Context.exit();
}
}

View File

@ -21,6 +21,7 @@ package com.metamx.druid.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs;
import com.metamx.druid.processing.ColumnSelectorFactory;
@ -44,6 +45,9 @@ public class LongSumAggregatorFactory implements AggregatorFactory
@JsonProperty("fieldName") final String fieldName
)
{
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
this.name = name;
this.fieldName = fieldName;
}

View File

@ -21,6 +21,7 @@ package com.metamx.druid.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Doubles;
import com.metamx.druid.processing.ColumnSelectorFactory;
@ -44,6 +45,9 @@ public class MaxAggregatorFactory implements AggregatorFactory
@JsonProperty("fieldName") final String fieldName
)
{
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
this.name = name;
this.fieldName = fieldName;
}

View File

@ -21,6 +21,7 @@ package com.metamx.druid.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Doubles;
import com.metamx.druid.processing.ColumnSelectorFactory;
@ -44,6 +45,9 @@ public class MinAggregatorFactory implements AggregatorFactory
@JsonProperty("fieldName") final String fieldName
)
{
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
this.name = name;
this.fieldName = fieldName;
}

View File

@ -143,15 +143,16 @@ public class DbConnector
@Override
public Void withHandle(Handle handle) throws Exception
{
List<Map<String, Object>> table = handle.select(String.format("SHOW tables LIKE '%s'", tableName));
if ( !handle.getConnection().getMetaData().getDatabaseProductName().contains("PostgreSQL") ) {
List<Map<String, Object>> table = handle.select(String.format("SHOW tables LIKE '%s'", tableName));
if (table.isEmpty()) {
log.info("Creating table[%s]", tableName);
handle.createStatement(sql).execute();
} else {
log.info("Table[%s] existed: [%s]", tableName, table);
if (table.isEmpty()) {
log.info("Creating table[%s]", tableName);
handle.createStatement(sql).execute();
} else {
log.info("Table[%s] existed: [%s]", tableName, table);
}
}
return null;
}
}

View File

@ -34,4 +34,10 @@ public class Runnables
}
};
}
public static Runnable getNoopRunnable(){
return new Runnable(){
public void run(){}
};
}
}

27
examples/bin/ec2/env.sh Normal file
View File

@ -0,0 +1,27 @@
# Setup Oracle Java
sudo apt-get update
sudo add-apt-repository -y ppa:webupd8team/java
sudo apt-get update
# Setup yes answer to license question
echo debconf shared/accepted-oracle-license-v1-1 select true | sudo debconf-set-selections
echo debconf shared/accepted-oracle-license-v1-1 seen true | sudo debconf-set-selections
sudo apt-get -y -q install oracle-java7-installer
# Automated Kafka setup
curl http://static.druid.io/artifacts/kafka-0.7.2-incubating-bin.tar.gz -o /tmp/kafka-0.7.2-incubating-bin.tar.gz
tar -xvzf /tmp/kafka-0.7.2-incubating-bin.tar.gz
cd kafka-0.7.2-incubating-bin
cat config/zookeeper.properties
nohup bin/zookeeper-server-start.sh config/zookeeper.properties 2>&1 > /dev/null &
# in a new console
nohup bin/kafka-server-start.sh config/server.properties 2>&1 > /dev/null &
# Install dependencies - mysql must be built from source, as the 12.04 apt-get hangs
export DEBIAN_FRONTEND=noninteractive
sudo debconf-set-selections <<< 'mysql-server-5.5 mysql-server/root_password password diurd'
sudo debconf-set-selections <<< 'mysql-server-5.5 mysql-server/root_password_again password diurd'
sudo apt-get -q -y -V --force-yes --reinstall install mysql-server-5.5
echo "ALL DONE with druid environment setup! Hit CTRL-C to proceed."
exit 0

22
examples/bin/ec2/run.sh Normal file
View File

@ -0,0 +1,22 @@
# Is localhost expected with multi-node?
mysql -u root -pdiurd -e "GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd'; CREATE database druid;" 2>&1 > /dev/null
tar -xvzf druid-services-*-bin.tar.gz 2>&1 > /dev/null
cd druid-services-* 2>&1 > /dev/null
mkdir logs 2>&1 > /dev/null
# Now start a realtime node
nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=config/realtime/realtime.spec -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/realtime com.metamx.druid.realtime.RealtimeMain 2>&1 > logs/realtime.log &
# And a master node
nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/master com.metamx.druid.http.MasterMain 2>&1 > logs/master.log &
# And a compute node
nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/compute com.metamx.druid.http.ComputeMain 2>&1 > logs/compute.log &
# And a broker node
nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/broker com.metamx.druid.http.BrokerMain 2>&1 > logs/broker.log &
echo "Hit CTRL-C to continue..."
exit 0

View File

@ -0,0 +1,12 @@
{
"queryType": "groupBy",
"dataSource": "rabbitmqtest",
"granularity": "all",
"dimensions": [],
"aggregations": [
{ "type": "count", "name": "rows" },
{"type": "longSum", "name": "imps", "fieldName": "impressions"},
{"type": "doubleSum", "name": "wp", "fieldName": "wp"}
],
"intervals": ["2010-01-01T00:00/2020-01-01T00:00"]
}

View File

@ -0,0 +1,44 @@
[{
"schema" : {
"dataSource":"rabbitmqtest",
"aggregators":[
{"type":"count", "name":"impressions"},
{"type":"doubleSum","name":"wp","fieldName":"wp"}
],
"indexGranularity":"minute",
"shardSpec" : { "type": "none" }
},
"config" : {
"maxRowsInMemory" : 500000,
"intermediatePersistPeriod" : "PT1m"
},
"firehose" : {
"type" : "rabbitmq",
"connection" : {
"host": "localhost",
"username": "test-dude",
"password": "word-dude",
"virtualHost": "test-vhost"
},
"config" : {
"exchange": "test-exchange",
"queue" : "druidtest",
"routingKey": "#",
"durable": "true",
"exclusive": "false",
"autoDelete": "false"
},
"parser" : {
"timestampSpec" : { "column" : "utcdt", "format" : "iso" },
"data" : { "format" : "json" },
"dimensionExclusions" : ["wp"]
}
},
"plumber" : {
"type" : "realtime",
"windowPeriod" : "PT5m",
"segmentGranularity":"hour",
"basePersistDirectory" : "/tmp/realtime/basePersist",
"rejectionPolicy": { "type": "messageTime" }
}
}]

View File

@ -13,10 +13,16 @@ if [ ! -e "$PF" ]; then
touch created
touch $PF
chmod 700 $PF
echo " Your twitter user login name and pw is needed "
read -p 'twitter username? ' TWIT_USER
read -s -p 'twitter password? ' TWIT_PW
echo "user=${TWIT_USER}" >> $PF
echo "password=${TWIT_PW}" >> $PF
TWIT_PW=""
echo " Your twitter OAuth information is needed. Go to https://twitter.com/oauth_clients/new to register a new application and retrieve your keys "
read -p 'Twitter consumer key? ' CONSUMER_KEY
read -p 'Twitter consumer secret? ' CONSUMER_SECRET
read -p 'Twitter access token? ' ACCESS_TOKEN
read -p 'Twitter access token secret? ' ACCESS_TOKEN_SECRET
echo "debug=true" >> $PF
echo "oauth.consumerKey=${CONSUMER_KEY}" >> $PF
echo "oauth.consumerSecret=${CONSUMER_SECRET}" >> $PF
echo "oauth.accessToken=${ACCESS_TOKEN}" >> $PF
echo "oauth.accessTokenSecret=${ACCESS_TOKEN_SECRET}" >> $PF
CONSUMER_SECRET=""
ACCESS_TOKEN_SECRET=""
fi

View File

@ -0,0 +1,27 @@
{
"queryType":"groupBy",
"dataSource":"webstream",
"granularity":"minute",
"dimensions":[
"timezone"
],
"aggregations":[
{
"type":"count",
"name":"rows"
},
{
"type":"doubleSum",
"fieldName":"known_users",
"name":"known_users"
}
],
"filter":{
"type":"selector",
"dimension":"country",
"value":"US"
},
"intervals":[
"2013-06-01T00:00/2020-01-01T00"
]
}

View File

@ -0,0 +1,47 @@
[{
"schema": {
"dataSource": "webstream",
"aggregators": [
{"type": "count", "name": "rows"},
{"type": "doubleSum", "fieldName": "known_users", "name": "known_users"}
],
"indexGranularity": "minute",
"shardSpec": {"type": "none"}
},
"config": {
"maxRowsInMemory": 50000,
"intermediatePersistPeriod": "PT2m"
},
"firehose": {
"type": "webstream",
"url":"http://developer.usa.gov/1usagov",
"renamedDimensions": {
"g":"bitly_hash",
"c":"country",
"a":"user",
"cy":"city",
"l":"encoding_user_login",
"hh":"short_url",
"hc":"timestamp_hash",
"h":"user_bitly_hash",
"u":"url",
"tz":"timezone",
"t":"time",
"r":"referring_url",
"gr":"geo_region",
"nk":"known_users",
"al":"accept_language"
},
"timeDimension":"t",
"timeFormat":"posix"
},
"plumber": {
"type": "realtime",
"windowPeriod": "PT3m",
"segmentGranularity": "hour",
"basePersistDirectory": "/tmp/example/usagov_realtime/basePersist"
}
}]

76
examples/bin/run_ec2.sh Executable file
View File

@ -0,0 +1,76 @@
# Before running, you will need to download the EC2 tools from http://aws.amazon.com/developertools/351
# and then setup your EC2_HOME and PATH variables (or similar):
#
# # Setup environment for ec2-api-tools
# export EC2_HOME=/path/to/ec2-api-tools-1.6.7.4/
# export PATH=$PATH:$EC2_HOME/bin
# export AWS_ACCESS_KEY=
# export AWS_SECRET_KEY=
# Check for ec2 commands we require and die if they're missing
type ec2-create-keypair >/dev/null 2>&1 || { echo >&2 "I require ec2-create-keypair but it's not installed. Aborting."; exit 1; }
type ec2-create-group >/dev/null 2>&1 || { echo >&2 "I require ec2-create-group but it's not installed. Aborting."; exit 1; }
type ec2-authorize >/dev/null 2>&1 || { echo >&2 "I require ec2-authorize but it's not installed. Aborting."; exit 1; }
type ec2-run-instances >/dev/null 2>&1 || { echo >&2 "I require ec2-run-instances but it's not installed. Aborting."; exit 1; }
type ec2-describe-instances >/dev/null 2>&1 || { echo >&2 "I require ec2-describe-instances but it's not installed. Aborting."; exit 1; }
# Create a keypair for our servers
echo "Removing old keypair for druid..."
ec2-delete-keypair druid-keypair
echo "Creating new keypair for druid..."
ec2-create-keypair druid-keypair > druid-keypair
chmod 0600 druid-keypair
mv druid-keypair ~/.ssh/
# Create a security group for our servers
echo "Creating a new security group for druid..."
ec2-create-group druid-group -d "Druid Cluster"
# Create rules that allow necessary services in our group
echo "Creating new firewall rules for druid..."
# SSH from outside
ec2-authorize druid-group -P tcp -p 22
# Enable all traffic within group
ec2-authorize druid-group -P tcp -p 1-65535 -o druid-group
ec2-authorize druid-group -P udp -p 1-65535 -o druid-group
echo "Booting a single small instance for druid..."
# Use ami ami-e7582d8e - Alestic Ubuntu 12.04 us-east
INSTANCE_ID=$(ec2-run-instances ami-e7582d8e -n 1 -g druid-group -k druid-keypair --instance-type m1.small| awk '/INSTANCE/{print $2}')
while true; do
sleep 1
INSTANCE_STATUS=$(ec2-describe-instances|grep INSTANCE|grep $INSTANCE_ID|cut -f6)
if [ $INSTANCE_STATUS == "running" ]
then
echo "Instance $INSTANCE_ID is status $INSTANCE_STATUS..."
break
fi
done
# Wait for the instance to come up
echo "Waiting 60 seconds for instance $INSTANCE_ID to boot..."
sleep 60
# Get hostname and ssh with the key we created, and ssh there
INSTANCE_ADDRESS=`ec2-describe-instances|grep 'INSTANCE'|grep $INSTANCE_ID|cut -f4`
echo "Connecting to $INSTANCE_ADDRESS to prepare environment for druid..."
scp -i ~/.ssh/druid-keypair -o StrictHostKeyChecking=no ./ec2/env.sh ubuntu@${INSTANCE_ADDRESS}:
ssh -q -f -i ~/.ssh/druid-keypair -o StrictHostKeyChecking=no ubuntu@${INSTANCE_ADDRESS} 'chmod +x ./env.sh;./env.sh'
echo "Prepared $INSTANCE_ADDRESS for druid."
# Now to scp a tarball up that can run druid!
if [ -f ../../services/target/druid-services-*-SNAPSHOT-bin.tar.gz ];
then
echo "Uploading druid tarball to server..."
scp -i ~/.ssh/druid-keypair -o StrictHostKeyChecking=no ../../services/target/druid-services-*-bin.tar.gz ubuntu@${INSTANCE_ADDRESS}:
else
echo "ERROR - package not built!"
fi
# Now boot druid parts
scp -i ~/.ssh/druid-keypair -o StrictHostKeyChecking=no ./ec2/run.sh ubuntu@${INSTANCE_ADDRESS}:
ssh -q -f -i ~/.ssh/druid-keypair -o StrictHostKeyChecking=no ubuntu@${INSTANCE_ADDRESS} 'chmod +x ./run.sh;./run.sh'
echo "Druid booting complete!"
echo "ssh -i ~/.ssh/druid-keypair ubuntu@${INSTANCE_ADDRESS} #to connect"

View File

@ -55,11 +55,11 @@ JAVA_ARGS="${JAVA_ARGS} -Ddruid.realtime.specFile=${SPEC_FILE}"
DRUID_CP=${EXAMPLE_LOC}
#For a pull
DRUID_CP=${DRUID_CP}:`ls ../target/druid-examples-*-selfcontained.jar`
DRUID_CP=${DRUID_CP}:../config
DRUID_CP=${DRUID_CP}:`ls ${SCRIPT_DIR}/../target/druid-examples-*-selfcontained.jar`
DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/../config
#For the kit
DRUID_CP=${DRUID_CP}:`ls ./lib/druid-examples-*-selfcontained.jar`
DRUID_CP=${DRUID_CP}:./config
DRUID_CP=${DRUID_CP}:`ls ${SCRIPT_DIR}/lib/druid-examples-*-selfcontained.jar`
DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/config
echo "Running command:"

View File

@ -0,0 +1,43 @@
druid.host=127.0.0.1
druid.port=8083
com.metamx.emitter.logging=true
druid.processing.formatString=processing_%s
druid.processing.numThreads=1
druid.processing.buffer.sizeBytes=10000000
#emitting, opaque marker
druid.service=example
druid.request.logging.dir=/tmp/example/log
druid.realtime.specFile=realtime.spec
com.metamx.emitter.logging=true
com.metamx.emitter.logging.level=info
# below are dummy values when operating a realtime only node
com.metamx.aws.accessKey=dummy_access_key
com.metamx.aws.secretKey=dummy_secret_key
druid.pusher.s3.bucket=dummy_s3_bucket
druid.zk.service.host=localhost
druid.server.maxSize=300000000000
druid.zk.paths.base=/druid
druid.database.segmentTable=prod_segments
druid.database.user=druid
druid.database.password=diurd
druid.database.connectURI=jdbc:mysql://localhost:3306/druid
druid.zk.paths.discoveryPath=/druid/discoveryPath
druid.database.ruleTable=rules
druid.database.configTable=config
# Path on local FS for storage of segments; dir will be created if needed
druid.paths.indexCache=/tmp/druid/indexCache
# Path on local FS for storage of segment metadata; dir will be created if needed
druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache
druid.pusher.local.storageDirectory=/tmp/druid/localStorage
druid.pusher.local=true
# thread pool size for servicing queries
druid.client.http.connections=30
druid.host=127.0.0.1:8083

View File

@ -0,0 +1,39 @@
druid.host=127.0.0.1
druid.port=8082
com.metamx.emitter.logging=true
druid.processing.formatString=processing_%s
druid.processing.numThreads=1
druid.processing.buffer.sizeBytes=10000000
#emitting, opaque marker
druid.service=example
druid.request.logging.dir=/tmp/example/log
druid.realtime.specFile=realtime.spec
com.metamx.emitter.logging=true
com.metamx.emitter.logging.level=info
# below are dummy values when operating a realtime only node
com.metamx.aws.accessKey=dummy_access_key
com.metamx.aws.secretKey=dummy_secret_key
druid.pusher.s3.bucket=dummy_s3_bucket
druid.zk.service.host=localhost
druid.server.maxSize=300000000000
druid.zk.paths.base=/druid
druid.database.segmentTable=prod_segments
druid.database.user=druid
druid.database.password=diurd
druid.database.connectURI=jdbc:mysql://localhost:3306/druid
druid.zk.paths.discoveryPath=/druid/discoveryPath
druid.database.ruleTable=rules
druid.database.configTable=config
# Path on local FS for storage of segments; dir will be created if needed
druid.paths.indexCache=/tmp/druid/indexCache
# Path on local FS for storage of segment metadata; dir will be created if needed
druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache
druid.pusher.local.storageDirectory=/tmp/druid/localStorage
druid.pusher.local=true

View File

@ -0,0 +1,39 @@
druid.host=127.0.0.1
druid.port=8081
com.metamx.emitter.logging=true
druid.processing.formatString=processing_%s
druid.processing.numThreads=1
druid.processing.buffer.sizeBytes=10000000
#emitting, opaque marker
druid.service=example
druid.request.logging.dir=/tmp/example/log
druid.realtime.specFile=realtime.spec
com.metamx.emitter.logging=true
com.metamx.emitter.logging.level=info
# below are dummy values when operating a realtime only node
com.metamx.aws.accessKey=dummy_access_key
com.metamx.aws.secretKey=dummy_secret_key
druid.pusher.s3.bucket=dummy_s3_bucket
druid.zk.service.host=localhost
druid.server.maxSize=300000000000
druid.zk.paths.base=/druid
druid.database.segmentTable=prod_segments
druid.database.user=druid
druid.database.password=diurd
druid.database.connectURI=jdbc:mysql://localhost:3306/druid
druid.zk.paths.discoveryPath=/druid/discoveryPath
druid.database.ruleTable=rules
druid.database.configTable=config
# Path on local FS for storage of segments; dir will be created if needed
druid.paths.indexCache=/tmp/druid/indexCache
# Path on local FS for storage of segment metadata; dir will be created if needed
druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache
druid.pusher.local.storageDirectory=/tmp/druid/localStorage
druid.pusher.local=true

View File

@ -0,0 +1,29 @@
[{
"schema" : { "dataSource":"druidtest",
"aggregators":[ {"type":"count", "name":"impressions"},
{"type":"doubleSum","name":"wp","fieldName":"wp"}],
"indexGranularity":"minute",
"shardSpec" : { "type": "none" } },
"config" : { "maxRowsInMemory" : 500000,
"intermediatePersistPeriod" : "PT10m" },
"firehose" : { "type" : "kafka-0.7.2",
"consumerProps" : { "zk.connect" : "localhost:2181",
"zk.connectiontimeout.ms" : "15000",
"zk.sessiontimeout.ms" : "15000",
"zk.synctime.ms" : "5000",
"groupid" : "topic-pixel-local",
"fetch.size" : "1048586",
"autooffset.reset" : "largest",
"autocommit.enable" : "false" },
"feed" : "druidtest",
"parser" : { "timestampSpec" : { "column" : "utcdt", "format" : "iso" },
"data" : { "format" : "json" },
"dimensionExclusions" : ["wp"] } },
"plumber" : { "type" : "realtime",
"windowPeriod" : "PT10m",
"segmentGranularity":"hour",
"basePersistDirectory" : "/tmp/realtime/basePersist",
"rejectionPolicy": {"type": "messageTime"} }
}]

View File

@ -0,0 +1,41 @@
druid.host=127.0.0.1
druid.port=8080
com.metamx.emitter.logging=true
druid.processing.formatString=processing_%s
druid.processing.numThreads=1
druid.processing.buffer.sizeBytes=10000000
#emitting, opaque marker
druid.service=example
druid.request.logging.dir=/tmp/example/log
druid.realtime.specFile=realtime.spec
com.metamx.emitter.logging=true
com.metamx.emitter.logging.level=info
# below are dummy values when operating a realtime only node
com.metamx.aws.accessKey=dummy_access_key
com.metamx.aws.secretKey=dummy_secret_key
druid.pusher.s3.bucket=dummy_s3_bucket
druid.zk.service.host=localhost
druid.server.maxSize=300000000000
druid.zk.paths.base=/druid
druid.database.segmentTable=prod_segments
druid.database.user=druid
druid.database.password=diurd
druid.database.connectURI=jdbc:mysql://localhost:3306/druid
druid.zk.paths.discoveryPath=/druid/discoveryPath
druid.database.ruleTable=rules
druid.database.configTable=config
# Path on local FS for storage of segments; dir will be created if needed
druid.paths.indexCache=/tmp/druid/indexCache
# Path on local FS for storage of segment metadata; dir will be created if needed
druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache
druid.pusher.local.storageDirectory=/tmp/druid/localStorage
druid.pusher.local=true
druid.host=127.0.0.1:8080

View File

@ -9,7 +9,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.5.0-SNAPSHOT</version>
<version>0.5.23-SNAPSHOT</version>
</parent>
<dependencies>
@ -32,17 +32,17 @@
<dependency>
<groupId>org.twitter4j</groupId>
<artifactId>twitter4j-core</artifactId>
<version>2.2.6</version>
<version>3.0.3</version>
</dependency>
<dependency>
<groupId>org.twitter4j</groupId>
<artifactId>twitter4j-async</artifactId>
<version>2.2.6</version>
<version>3.0.3</version>
</dependency>
<dependency>
<groupId>org.twitter4j</groupId>
<artifactId>twitter4j-stream</artifactId>
<version>2.2.6</version>
<version>3.0.3</version>
</dependency>
</dependencies>

View File

@ -16,6 +16,7 @@ import com.metamx.druid.realtime.SegmentPublisher;
import druid.examples.flights.FlightsFirehoseFactory;
import druid.examples.rand.RandomFirehoseFactory;
import druid.examples.twitter.TwitterSpritzerFirehoseFactory;
import druid.examples.web.WebFirehoseFactory;
import java.io.File;
import java.io.IOException;
@ -41,7 +42,9 @@ public class RealtimeStandaloneMain
rn.registerJacksonSubtype(
new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"),
new NamedType(FlightsFirehoseFactory.class, "flights"),
new NamedType(RandomFirehoseFactory.class, "rand")
new NamedType(RandomFirehoseFactory.class, "rand"),
new NamedType(WebFirehoseFactory.class, "webstream")
);
// Create dummy objects for the various interfaces that interact with the DB, ZK and deep storage
@ -50,7 +53,7 @@ public class RealtimeStandaloneMain
rn.setDataSegmentPusher(new NoopDataSegmentPusher());
rn.setServerView(new NoopServerView());
rn.setInventoryView(new NoopInventoryView());
Runtime.getRuntime().addShutdownHook(
new Thread(
new Runnable()
@ -142,5 +145,17 @@ public class RealtimeStandaloneMain
{
// do nothing
}
@Override
public void announceSegments(Iterable<DataSegment> segments) throws IOException
{
// do nothing
}
@Override
public void unannounceSegments(Iterable<DataSegment> segments) throws IOException
{
// do nothing
}
}
}

View File

@ -0,0 +1,182 @@
package druid.examples.rabbitmq;
import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
import com.rabbitmq.client.ConnectionFactory;
import org.apache.commons.cli.*;
import java.text.SimpleDateFormat;
import java.util.*;
/**
*
*/
public class RabbitMQProducerMain
{
public static void main(String[] args)
throws Exception
{
// We use a List to keep track of option insertion order. See below.
final List<Option> optionList = new ArrayList<Option>();
optionList.add(OptionBuilder.withLongOpt("help")
.withDescription("display this help message")
.create("h"));
optionList.add(OptionBuilder.withLongOpt("hostname")
.hasArg()
.withDescription("the hostname of the AMQP broker [defaults to AMQP library default]")
.create("b"));
optionList.add(OptionBuilder.withLongOpt("port")
.hasArg()
.withDescription("the port of the AMQP broker [defaults to AMQP library default]")
.create("n"));
optionList.add(OptionBuilder.withLongOpt("username")
.hasArg()
.withDescription("username to connect to the AMQP broker [defaults to AMQP library default]")
.create("u"));
optionList.add(OptionBuilder.withLongOpt("password")
.hasArg()
.withDescription("password to connect to the AMQP broker [defaults to AMQP library default]")
.create("p"));
optionList.add(OptionBuilder.withLongOpt("vhost")
.hasArg()
.withDescription("name of virtual host on the AMQP broker [defaults to AMQP library default]")
.create("v"));
optionList.add(OptionBuilder.withLongOpt("exchange")
.isRequired()
.hasArg()
.withDescription("name of the AMQP exchange [required - no default]")
.create("e"));
optionList.add(OptionBuilder.withLongOpt("key")
.hasArg()
.withDescription("the routing key to use when sending messages [default: 'default.routing.key']")
.create("k"));
optionList.add(OptionBuilder.withLongOpt("type")
.hasArg()
.withDescription("the type of exchange to create [default: 'topic']")
.create("t"));
optionList.add(OptionBuilder.withLongOpt("durable")
.withDescription("if set, a durable exchange will be declared [default: not set]")
.create("d"));
optionList.add(OptionBuilder.withLongOpt("autodelete")
.withDescription("if set, an auto-delete exchange will be declared [default: not set]")
.create("a"));
optionList.add(OptionBuilder.withLongOpt("single")
.withDescription("if set, only a single message will be sent [default: not set]")
.create("s"));
optionList.add(OptionBuilder.withLongOpt("start")
.hasArg()
.withDescription("time to use to start sending messages from [default: 2010-01-01T00:00:00]")
.create());
optionList.add(OptionBuilder.withLongOpt("stop")
.hasArg()
.withDescription("time to use to send messages until (format: '2013-07-18T23:45:59') [default: current time]")
.create());
optionList.add(OptionBuilder.withLongOpt("interval")
.hasArg()
.withDescription("the interval to add to the timestamp between messages in seconds [default: 10]")
.create());
optionList.add(OptionBuilder.withLongOpt("delay")
.hasArg()
.withDescription("the delay between sending messages in milliseconds [default: 100]")
.create());
// An extremely silly hack to maintain the above order in the help formatting.
HelpFormatter formatter = new HelpFormatter();
// Add a comparator to the HelpFormatter using the ArrayList above to sort by insertion order.
formatter.setOptionComparator(new Comparator(){
@Override
public int compare(Object o1, Object o2)
{
// I know this isn't fast, but who cares! The list is short.
return optionList.indexOf(o1) - optionList.indexOf(o2);
}
});
// Now we can add all the options to an Options instance. This is dumb!
Options options = new Options();
for (Option option : optionList) {
options.addOption(option);
}
CommandLine cmd = null;
try{
cmd = new BasicParser().parse(options, args);
}
catch(ParseException e){
formatter.printHelp("RabbitMQProducerMain", e.getMessage(), options, null);
System.exit(1);
}
if(cmd.hasOption("h")) {
formatter.printHelp("RabbitMQProducerMain", options);
System.exit(2);
}
ConnectionFactory factory = new ConnectionFactory();
if(cmd.hasOption("b")){
factory.setHost(cmd.getOptionValue("b"));
}
if(cmd.hasOption("u")){
factory.setUsername(cmd.getOptionValue("u"));
}
if(cmd.hasOption("p")){
factory.setPassword(cmd.getOptionValue("p"));
}
if(cmd.hasOption("v")){
factory.setVirtualHost(cmd.getOptionValue("v"));
}
if(cmd.hasOption("n")){
factory.setPort(Integer.parseInt(cmd.getOptionValue("n")));
}
String exchange = cmd.getOptionValue("e");
String routingKey = "default.routing.key";
if(cmd.hasOption("k")){
routingKey = cmd.getOptionValue("k");
}
boolean durable = cmd.hasOption("d");
boolean autoDelete = cmd.hasOption("a");
String type = cmd.getOptionValue("t", "topic");
boolean single = cmd.hasOption("single");
int interval = Integer.parseInt(cmd.getOptionValue("interval", "10"));
int delay = Integer.parseInt(cmd.getOptionValue("delay", "100"));
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
Date stop = sdf.parse(cmd.getOptionValue("stop", sdf.format(new Date())));
Random r = new Random();
Calendar timer = Calendar.getInstance();
timer.setTime(sdf.parse(cmd.getOptionValue("start", "2010-01-01T00:00:00")));
String msg_template = "{\"utcdt\": \"%s\", \"wp\": %d, \"gender\": \"%s\", \"age\": %d}";
Connection connection = factory.newConnection();
Channel channel = connection.createChannel();
channel.exchangeDeclare(exchange, type, durable, autoDelete, null);
do{
int wp = (10 + r.nextInt(90)) * 100;
String gender = r.nextBoolean() ? "male" : "female";
int age = 20 + r.nextInt(70);
String line = String.format(msg_template, sdf.format(timer.getTime()), wp, gender, age);
channel.basicPublish(exchange, routingKey, null, line.getBytes());
System.out.println("Sent message: " + line);
timer.add(Calendar.SECOND, interval);
Thread.sleep(delay);
}while((!single && stop.after(timer.getTime())));
connection.close();
}
}

View File

@ -17,6 +17,7 @@ import twitter4j.StatusListener;
import twitter4j.TwitterStream;
import twitter4j.TwitterStreamFactory;
import twitter4j.User;
import twitter4j.StallWarning;
import java.io.IOException;
import java.util.Arrays;
@ -31,7 +32,7 @@ import java.util.concurrent.TimeUnit;
import static java.lang.Thread.sleep;
/**
* Twitter "spritzer" Firehost Factory named "twitzer".
* Twitter "spritzer" Firehose Factory named "twitzer".
* Builds a Firehose that emits a stream of
* ??
* with timestamps along with ??.
@ -176,6 +177,11 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory {
{
ex.printStackTrace();
}
@Override
public void onStallWarning(StallWarning warning) {
System.out.println("Got stall warning:" + warning);
}
};
twitterStream.addListener(statusListener);

View File

@ -0,0 +1,133 @@
/*
* 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 druid.examples.web;
import com.fasterxml.jackson.core.JsonParseException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.JsonMappingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.io.InputSupplier;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.emitter.EmittingLogger;
import java.io.BufferedReader;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.TimeUnit;
public class InputSupplierUpdateStream implements UpdateStream
{
private static final EmittingLogger log = new EmittingLogger(InputSupplierUpdateStream.class);
private static final long queueWaitTime = 15L;
private final TypeReference<HashMap<String, Object>> typeRef;
private final InputSupplier<BufferedReader> supplier;
private final int QUEUE_SIZE = 10000;
private final BlockingQueue<Map<String, Object>> queue = new ArrayBlockingQueue<Map<String, Object>>(QUEUE_SIZE);
private final ObjectMapper mapper = new DefaultObjectMapper();
private final String timeDimension;
private final Thread addToQueueThread;
public InputSupplierUpdateStream(
final InputSupplier<BufferedReader> supplier,
final String timeDimension
)
{
addToQueueThread = new Thread()
{
public void run()
{
while (!isInterrupted()) {
try {
BufferedReader reader = supplier.getInput();
String line;
while ((line = reader.readLine()) != null) {
if (isValid(line)) {
HashMap<String, Object> map = mapper.readValue(line, typeRef);
if (map.get(timeDimension) != null) {
queue.offer(map, queueWaitTime, TimeUnit.SECONDS);
log.debug("Successfully added to queue");
} else {
log.info("missing timestamp");
}
}
}
}
catch (InterruptedException e){
log.info(e, "Thread adding events to the queue interrupted");
return;
}
catch (JsonMappingException e) {
log.info(e, "Error in converting json to map");
}
catch (JsonParseException e) {
log.info(e, "Error in parsing json");
}
catch (IOException e) {
log.info(e, "Error in connecting to InputStream");
}
}
}
};
addToQueueThread.setDaemon(true);
this.supplier = supplier;
this.typeRef = new TypeReference<HashMap<String, Object>>()
{
};
this.timeDimension = timeDimension;
}
private boolean isValid(String s)
{
return !(s.isEmpty());
}
public void start()
{
addToQueueThread.start();
}
public void stop()
{
addToQueueThread.interrupt();
}
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
{
return queue.poll(waitTime, unit);
}
public int getQueueSize()
{
return queue.size();
}
public String getTimeDimension()
{
return timeDimension;
}
}

View File

@ -0,0 +1,42 @@
/*
* 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 druid.examples.web;
import com.google.common.io.InputSupplier;
import java.io.BufferedReader;
public class InputSupplierUpdateStreamFactory implements UpdateStreamFactory
{
private final InputSupplier<BufferedReader> inputSupplier;
private final String timeDimension;
public InputSupplierUpdateStreamFactory(InputSupplier<BufferedReader> inputSupplier, String timeDimension)
{
this.inputSupplier = inputSupplier;
this.timeDimension = timeDimension;
}
public InputSupplierUpdateStream build()
{
return new InputSupplierUpdateStream(inputSupplier, timeDimension);
}
}

View File

@ -0,0 +1,81 @@
/*
* 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 druid.examples.web;
import com.google.common.collect.Maps;
import java.util.Map;
import java.util.concurrent.TimeUnit;
public class RenamingKeysUpdateStream implements UpdateStream
{
private final InputSupplierUpdateStream updateStream;
private Map<String, String> renamedDimensions;
public RenamingKeysUpdateStream(
InputSupplierUpdateStream updateStream,
Map<String, String> renamedDimensions
)
{
this.renamedDimensions = renamedDimensions;
this.updateStream = updateStream;
}
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
{
return renameKeys(updateStream.pollFromQueue(waitTime, unit));
}
private Map<String, Object> renameKeys(Map<String, Object> update)
{
if (renamedDimensions != null) {
Map<String, Object> renamedMap = Maps.newHashMap();
for (String key : renamedDimensions.keySet()) {
if (update.get(key) != null) {
Object obj = update.get(key);
renamedMap.put(renamedDimensions.get(key), obj);
}
}
return renamedMap;
} else {
return update;
}
}
public String getTimeDimension()
{
if (renamedDimensions != null && renamedDimensions.get(updateStream.getTimeDimension()) != null) {
return renamedDimensions.get(updateStream.getTimeDimension());
}
return updateStream.getTimeDimension();
}
public void start()
{
updateStream.start();
}
public void stop(){
updateStream.stop();
}
}

View File

@ -0,0 +1,39 @@
/*
* 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 druid.examples.web;
import java.util.Map;
public class RenamingKeysUpdateStreamFactory implements UpdateStreamFactory
{
private InputSupplierUpdateStreamFactory updateStreamFactory;
private Map<String, String> renamedDimensions;
public RenamingKeysUpdateStreamFactory(InputSupplierUpdateStreamFactory updateStreamFactory, Map<String, String> renamedDimensions)
{
this.updateStreamFactory = updateStreamFactory;
this.renamedDimensions = renamedDimensions;
}
public RenamingKeysUpdateStream build()
{
return new RenamingKeysUpdateStream(updateStreamFactory.build(), renamedDimensions);
}
}

View File

@ -0,0 +1,31 @@
/*
* 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 druid.examples.web;
import java.util.Map;
import java.util.concurrent.TimeUnit;
public interface UpdateStream
{
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException;
public String getTimeDimension();
public void start();
public void stop();
}

View File

@ -0,0 +1,24 @@
/*
* 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 druid.examples.web;
public interface UpdateStreamFactory
{
public UpdateStream build();
}

View File

@ -0,0 +1,134 @@
/*
* 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 druid.examples.web;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Throwables;
import com.metamx.common.parsers.TimestampParser;
import com.metamx.druid.guava.Runnables;
import com.metamx.druid.input.InputRow;
import com.metamx.druid.input.MapBasedInputRow;
import com.metamx.druid.realtime.firehose.Firehose;
import com.metamx.druid.realtime.firehose.FirehoseFactory;
import com.metamx.emitter.EmittingLogger;
import org.joda.time.DateTime;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Map;
import java.util.concurrent.TimeUnit;
@JsonTypeName("webstream")
public class WebFirehoseFactory implements FirehoseFactory
{
private static final EmittingLogger log = new EmittingLogger(WebFirehoseFactory.class);
private final String timeFormat;
private final UpdateStreamFactory factory;
private final long queueWaitTime = 15L;
@JsonCreator
public WebFirehoseFactory(
@JsonProperty("url") String url,
@JsonProperty("renamedDimensions") Map<String, String> renamedDimensions,
@JsonProperty("timeDimension") String timeDimension,
@JsonProperty("timeFormat") String timeFormat
)
{
this(
new RenamingKeysUpdateStreamFactory(
new InputSupplierUpdateStreamFactory(new WebJsonSupplier(url), timeDimension),
renamedDimensions
), timeFormat
);
}
public WebFirehoseFactory(UpdateStreamFactory factory, String timeFormat)
{
this.factory = factory;
if (timeFormat == null) {
this.timeFormat = "auto";
} else {
this.timeFormat = timeFormat;
}
}
@Override
public Firehose connect() throws IOException
{
final UpdateStream updateStream = factory.build();
updateStream.start();
return new Firehose()
{
Map<String, Object> map;
private final Runnable doNothingRunnable = Runnables.getNoopRunnable();
@Override
public boolean hasMore()
{
try {
map = updateStream.pollFromQueue(queueWaitTime, TimeUnit.SECONDS);
return map != null;
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
}
}
@Override
public InputRow nextRow()
{
try {
DateTime date = TimestampParser.createTimestampParser(timeFormat)
.apply(map.get(updateStream.getTimeDimension()).toString());
return new MapBasedInputRow(
date.getMillis(),
new ArrayList(map.keySet()),
map
);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
finally {
map = null;
}
}
@Override
public Runnable commit()
{
// ephemera in, ephemera out.
return doNothingRunnable; // reuse the same object each time
}
@Override
public void close() throws IOException
{
updateStream.stop();
}
};
}
}

View File

@ -0,0 +1,57 @@
/*
* 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 druid.examples.web;
import com.google.common.io.InputSupplier;
import com.metamx.emitter.EmittingLogger;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.net.URL;
import java.net.URLConnection;
public class WebJsonSupplier implements InputSupplier<BufferedReader>
{
private static final EmittingLogger log = new EmittingLogger(WebJsonSupplier.class);
private String urlString;
private URL url;
public WebJsonSupplier(String urlString)
{
this.urlString = urlString;
try {
this.url = new URL(urlString);
}
catch (Exception e) {
log.error(e,"Malformed url");
}
}
@Override
public BufferedReader getInput() throws IOException
{
URL url = new URL(urlString);
URLConnection connection = url.openConnection();
connection.setDoInput(true);
return new BufferedReader(new InputStreamReader(url.openStream()));
}
}

View File

@ -0,0 +1,114 @@
/*
* 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 druid.examples.web;
import com.google.common.io.InputSupplier;
import junit.framework.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
public class InputSupplierUpdateStreamTest
{
private final long waitTime = 1L;
private final TimeUnit unit = TimeUnit.SECONDS;
private final ArrayList<String> dimensions = new ArrayList<String>();
private InputSupplier testCaseSupplier;
Map<String, Object> expectedAnswer = new HashMap<String, Object>();
String timeDimension;
@Before
public void setUp()
{
timeDimension = "time";
testCaseSupplier = new TestCaseSupplier(
"{\"item1\": \"value1\","
+ "\"item2\":2,"
+ "\"time\":1372121562 }"
);
dimensions.add("item1");
dimensions.add("item2");
dimensions.add("time");
expectedAnswer.put("item1", "value1");
expectedAnswer.put("item2", 2);
expectedAnswer.put("time", 1372121562);
}
@Test
public void basicIngestionCheck() throws Exception
{
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(
testCaseSupplier,
timeDimension
);
updateStream.start();
Map<String, Object> insertedRow = updateStream.pollFromQueue(waitTime, unit);
Assert.assertEquals(expectedAnswer, insertedRow);
updateStream.stop();
}
//If a timestamp is missing, we should throw away the event
@Test
public void missingTimeStampCheck()
{
testCaseSupplier = new TestCaseSupplier(
"{\"item1\": \"value1\","
+ "\"item2\":2}"
);
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(
testCaseSupplier,
timeDimension
);
updateStream.start();
Assert.assertEquals(updateStream.getQueueSize(), 0);
updateStream.stop();
}
//If any other value is missing, we should still add the event and process it properly
@Test
public void otherNullValueCheck() throws Exception
{
testCaseSupplier = new TestCaseSupplier(
"{\"item1\": \"value1\","
+ "\"time\":1372121562 }"
);
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(
testCaseSupplier,
timeDimension
);
updateStream.start();
Map<String, Object> insertedRow = updateStream.pollFromQueue(waitTime, unit);
Map<String, Object> expectedAnswer = new HashMap<String, Object>();
expectedAnswer.put("item1", "value1");
expectedAnswer.put("time", 1372121562);
Assert.assertEquals(expectedAnswer, insertedRow);
updateStream.stop();
}
}

View File

@ -0,0 +1,91 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package druid.examples.web;
import com.google.common.io.InputSupplier;
import junit.framework.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
public class RenamingKeysUpdateStreamTest
{
private final long waitTime = 15L;
private final TimeUnit unit = TimeUnit.SECONDS;
private InputSupplier testCaseSupplier;
String timeDimension;
@Before
public void setUp()
{
timeDimension = "time";
testCaseSupplier = new TestCaseSupplier(
"{\"item1\": \"value1\","
+ "\"item2\":2,"
+ "\"time\":1372121562 }"
);
}
@Test
public void testPolFromQueue() throws Exception
{
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(testCaseSupplier, timeDimension);
Map<String, String> renamedKeys = new HashMap<String, String>();
renamedKeys.put("item1", "i1");
renamedKeys.put("item2", "i2");
renamedKeys.put("time", "t");
RenamingKeysUpdateStream renamer = new RenamingKeysUpdateStream(updateStream, renamedKeys);
renamer.start();
Map<String, Object> expectedAnswer = new HashMap<String, Object>();
expectedAnswer.put("i1", "value1");
expectedAnswer.put("i2", 2);
expectedAnswer.put("t", 1372121562);
Assert.assertEquals(expectedAnswer, renamer.pollFromQueue(waitTime, unit));
}
@Test
public void testGetTimeDimension() throws Exception
{
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(testCaseSupplier, timeDimension);
Map<String, String> renamedKeys = new HashMap<String, String>();
renamedKeys.put("item1", "i1");
renamedKeys.put("item2", "i2");
renamedKeys.put("time", "t");
RenamingKeysUpdateStream renamer = new RenamingKeysUpdateStream(updateStream, renamedKeys);
Assert.assertEquals("t", renamer.getTimeDimension());
}
@Test
public void testMissingTimeRename() throws Exception
{
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(testCaseSupplier, timeDimension);
Map<String, String> renamedKeys = new HashMap<String, String>();
renamedKeys.put("item1", "i1");
renamedKeys.put("item2", "i2");
RenamingKeysUpdateStream renamer = new RenamingKeysUpdateStream(updateStream, renamedKeys);
Assert.assertEquals("time", renamer.getTimeDimension());
}
}

View File

@ -0,0 +1,42 @@
/*
* 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 druid.examples.web;
import com.google.common.io.InputSupplier;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.StringReader;
public class TestCaseSupplier implements InputSupplier<BufferedReader>
{
private final String testString;
public TestCaseSupplier(String testString)
{
this.testString = testString;
}
@Override
public BufferedReader getInput() throws IOException
{
return new BufferedReader(new StringReader(testString));
}
}

View File

@ -0,0 +1,223 @@
/*
* 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 druid.examples.web;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.metamx.druid.input.InputRow;
import com.metamx.druid.realtime.firehose.Firehose;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
public class WebFirehoseFactoryTest
{
private List<String> dimensions = Lists.newArrayList();
private WebFirehoseFactory webbie;
private WebFirehoseFactory webbie1;
@Before
public void setUp() throws Exception
{
dimensions.add("item1");
dimensions.add("item2");
dimensions.add("time");
webbie = new WebFirehoseFactory(
new UpdateStreamFactory()
{
@Override
public UpdateStream build()
{
return new MyUpdateStream(ImmutableMap.<String,Object>of("item1", "value1", "item2", 2, "time", "1372121562"));
}
},
"posix"
);
webbie1 = new WebFirehoseFactory(
new UpdateStreamFactory()
{
@Override
public UpdateStream build()
{
return new MyUpdateStream(ImmutableMap.<String,Object>of("item1", "value1", "item2", 2, "time", "1373241600000"));
}
},
"auto"
);
}
@Test
public void testDimensions() throws Exception
{
InputRow inputRow;
Firehose firehose = webbie.connect();
if (firehose.hasMore()) {
inputRow = firehose.nextRow();
} else {
throw new RuntimeException("queue is empty");
}
List<String> actualAnswer = inputRow.getDimensions();
Collections.sort(actualAnswer);
Assert.assertEquals(actualAnswer, dimensions);
}
@Test
public void testPosixTimeStamp() throws Exception
{
InputRow inputRow;
Firehose firehose = webbie.connect();
if (firehose.hasMore()) {
inputRow = firehose.nextRow();
} else {
throw new RuntimeException("queue is empty");
}
long expectedTime = 1372121562L * 1000L;
Assert.assertEquals(expectedTime, inputRow.getTimestampFromEpoch());
}
@Test
public void testISOTimeStamp() throws Exception
{
WebFirehoseFactory webbie3 = new WebFirehoseFactory(
new UpdateStreamFactory()
{
@Override
public UpdateStream build()
{
return new MyUpdateStream(ImmutableMap.<String,Object>of("item1", "value1", "item2", 2, "time", "2013-07-08"));
}
},
"auto"
);
Firehose firehose1 = webbie3.connect();
if (firehose1.hasMore()) {
long milliSeconds = firehose1.nextRow().getTimestampFromEpoch();
DateTime date = new DateTime("2013-07-08");
Assert.assertEquals(date.getMillis(), milliSeconds);
} else {
Assert.assertFalse("hasMore returned false", true);
}
}
@Test
public void testAutoIsoTimeStamp() throws Exception
{
WebFirehoseFactory webbie2 = new WebFirehoseFactory(
new UpdateStreamFactory()
{
@Override
public UpdateStream build()
{
return new MyUpdateStream(ImmutableMap.<String,Object>of("item1", "value1", "item2", 2, "time", "2013-07-08"));
}
},
null
);
Firehose firehose2 = webbie2.connect();
if (firehose2.hasMore()) {
long milliSeconds = firehose2.nextRow().getTimestampFromEpoch();
DateTime date = new DateTime("2013-07-08");
Assert.assertEquals(date.getMillis(), milliSeconds);
} else {
Assert.assertFalse("hasMore returned false", true);
}
}
@Test
public void testAutoMilliSecondsTimeStamp() throws Exception
{
Firehose firehose3 = webbie1.connect();
if (firehose3.hasMore()) {
long milliSeconds = firehose3.nextRow().getTimestampFromEpoch();
DateTime date = new DateTime("2013-07-08");
Assert.assertEquals(date.getMillis(), milliSeconds);
} else {
Assert.assertFalse("hasMore returned false", true);
}
}
@Test
public void testGetDimension() throws Exception
{
InputRow inputRow;
Firehose firehose = webbie1.connect();
if (firehose.hasMore()) {
inputRow = firehose.nextRow();
} else {
throw new RuntimeException("queue is empty");
}
List<String> column1 = Lists.newArrayList();
column1.add("value1");
Assert.assertEquals(column1, inputRow.getDimension("item1"));
}
@Test
public void testGetFloatMetric() throws Exception
{
InputRow inputRow;
Firehose firehose = webbie1.connect();
if (firehose.hasMore()) {
inputRow = firehose.nextRow();
} else {
throw new RuntimeException("queue is empty");
}
Assert.assertEquals((float) 2.0, inputRow.getFloatMetric("item2"), 0.0f);
}
private static class MyUpdateStream implements UpdateStream
{
private static ImmutableMap<String,Object> map;
public MyUpdateStream(ImmutableMap<String,Object> map){
this.map=map;
}
@Override
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
{
return map;
}
@Override
public String getTimeDimension()
{
return "time";
}
@Override
public void start()
{
}
@Override
public void stop()
{
}
}
}

View File

@ -0,0 +1,36 @@
/*
* 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 druid.examples.web;
import org.junit.Test;
import java.io.IOException;
public class WebJsonSupplierTest
{
@Test(expected = IOException.class)
public void checkInvalidUrl() throws Exception
{
String invalidURL = "http://invalid.url";
WebJsonSupplier supplier = new WebJsonSupplier(invalidURL);
supplier.getInput();
}
}

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.5.0-SNAPSHOT</version>
<version>0.5.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -19,15 +19,14 @@
package com.metamx.druid.common.s3;
import com.google.common.base.Throwables;
import com.metamx.common.logger.Logger;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.model.S3Bucket;
import org.jets3t.service.ServiceException;
import org.jets3t.service.model.S3Object;
import java.io.File;
import java.io.IOException;
import java.security.NoSuchAlgorithmException;
import java.util.Random;
import java.util.concurrent.Callable;
/**
*
@ -36,37 +35,6 @@ public class S3Utils
{
private static final Logger log = new Logger(S3Utils.class);
public static void putFileToS3(
File localFile, RestS3Service s3Client, String outputS3Bucket, String outputS3Path
)
throws S3ServiceException, IOException, NoSuchAlgorithmException
{
S3Object s3Obj = new S3Object(localFile);
s3Obj.setBucketName(outputS3Bucket);
s3Obj.setKey(outputS3Path);
log.info("Uploading file[%s] to [s3://%s/%s]", localFile, s3Obj.getBucketName(), s3Obj.getKey());
s3Client.putObject(new S3Bucket(outputS3Bucket), s3Obj);
}
public static void putFileToS3WrapExceptions(
File localFile, RestS3Service s3Client, String outputS3Bucket, String outputS3Path
)
{
try {
putFileToS3(localFile, s3Client, outputS3Bucket, outputS3Path);
}
catch (S3ServiceException e) {
throw new RuntimeException(e);
}
catch (IOException e) {
throw new RuntimeException(e);
}
catch (NoSuchAlgorithmException e) {
throw new RuntimeException(e);
}
}
public static void closeStreamsQuietly(S3Object s3Obj)
{
if (s3Obj == null) {
@ -80,4 +48,52 @@ public class S3Utils
}
}
/**
* Retries S3 operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not
* found, etc) are not retried.
*/
public static <T> T retryS3Operation(Callable<T> f) throws ServiceException, InterruptedException
{
int nTry = 0;
final int maxTries = 3;
while (true) {
try {
nTry++;
return f.call();
}
catch (IOException e) {
if (nTry <= maxTries) {
awaitNextRetry(e, nTry);
} else {
throw Throwables.propagate(e);
}
}
catch (ServiceException e) {
if (nTry <= maxTries &&
(e.getCause() instanceof IOException ||
(e.getErrorCode() != null && e.getErrorCode().equals("RequestTimeout")))) {
awaitNextRetry(e, nTry);
} else {
throw e;
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
private static void awaitNextRetry(Exception e, int nTry) throws InterruptedException
{
final long baseSleepMillis = 1000;
final double fuzziness = 0.2;
final long sleepMillis = Math.max(
baseSleepMillis,
(long) (baseSleepMillis * Math.pow(2, nTry) *
(1 + new Random().nextGaussian() * fuzziness))
);
log.info(e, "S3 fail on try %d, retrying in %,dms.", nTry, sleepMillis);
Thread.sleep(sleepMillis);
}
}

View File

@ -23,6 +23,9 @@ import com.metamx.druid.index.column.ColumnSelector;
import com.metamx.druid.kv.Indexed;
import org.joda.time.Interval;
import java.io.Closeable;
import java.io.IOException;
/**
*/
public interface QueryableIndex extends ColumnSelector
@ -31,4 +34,11 @@ public interface QueryableIndex extends ColumnSelector
public int getNumRows();
public Indexed<String> getColumnNames();
public Indexed<String> getAvailableDimensions();
/**
* The close method shouldn't actually be here as this is nasty. We will adjust it in the future.
* @throws IOException
*/
@Deprecated
public void close() throws IOException;
}

View File

@ -19,10 +19,12 @@
package com.metamx.druid.index;
import com.metamx.common.io.smoosh.SmooshedFileMapper;
import com.metamx.druid.index.column.Column;
import com.metamx.druid.kv.Indexed;
import org.joda.time.Interval;
import java.io.IOException;
import java.util.Map;
/**
@ -34,13 +36,15 @@ public class SimpleQueryableIndex implements QueryableIndex
private final Indexed<String> availableDimensions;
private final Column timeColumn;
private final Map<String, Column> otherColumns;
private final SmooshedFileMapper fileMapper;
public SimpleQueryableIndex(
Interval dataInterval,
Indexed<String> columnNames,
Indexed<String> dimNames,
Column timeColumn,
Map<String, Column> otherColumns
Map<String, Column> otherColumns,
SmooshedFileMapper fileMapper
)
{
this.dataInterval = dataInterval;
@ -48,6 +52,7 @@ public class SimpleQueryableIndex implements QueryableIndex
this.availableDimensions = dimNames;
this.timeColumn = timeColumn;
this.otherColumns = otherColumns;
this.fileMapper = fileMapper;
}
@Override
@ -85,4 +90,10 @@ public class SimpleQueryableIndex implements QueryableIndex
{
return otherColumns.get(columnName);
}
@Override
public void close() throws IOException
{
fileMapper.close();
}
}

View File

@ -375,7 +375,8 @@ public class IndexIO
dimValueLookups,
dimColumns,
invertedIndexed,
spatialIndexed
spatialIndexed,
smooshedFiles
);
log.debug("Mapped v8 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);
@ -761,7 +762,8 @@ public class IndexIO
.setType(ValueType.LONG)
.setGenericColumn(new LongGenericColumnSupplier(index.timestamps))
.build(),
columns
columns,
index.getFileMapper()
);
}
}
@ -795,7 +797,7 @@ public class IndexIO
}
final QueryableIndex index = new SimpleQueryableIndex(
dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time")), columns
dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time")), columns, smooshedFiles
);
log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);

View File

@ -24,18 +24,19 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.common.io.smoosh.SmooshedFileMapper;
import com.metamx.common.logger.Logger;
import com.metamx.druid.kv.ConciseCompressedIndexedInts;
import com.metamx.druid.kv.GenericIndexed;
import com.metamx.druid.kv.Indexed;
import com.metamx.druid.kv.IndexedList;
import com.metamx.druid.kv.IndexedLongs;
import com.metamx.druid.kv.IndexedRTree;
import com.metamx.druid.kv.VSizeIndexed;
import com.metamx.druid.kv.VSizeIndexedInts;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import org.joda.time.Interval;
import java.io.IOException;
import java.nio.ByteOrder;
import java.nio.LongBuffer;
import java.util.Arrays;
@ -57,6 +58,7 @@ public class MMappedIndex
final Map<String, VSizeIndexed> dimColumns;
final Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes;
final Map<String, ImmutableRTree> spatialIndexes;
final SmooshedFileMapper fileMapper;
private final Map<String, Integer> metricIndexes = Maps.newHashMap();
@ -69,7 +71,8 @@ public class MMappedIndex
Map<String, GenericIndexed<String>> dimValueLookups,
Map<String, VSizeIndexed> dimColumns,
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes,
Map<String, ImmutableRTree> spatialIndexes
Map<String, ImmutableRTree> spatialIndexes,
SmooshedFileMapper fileMapper
)
{
this.availableDimensions = availableDimensions;
@ -81,6 +84,7 @@ public class MMappedIndex
this.dimColumns = dimColumns;
this.invertedIndexes = invertedIndexes;
this.spatialIndexes = spatialIndexes;
this.fileMapper = fileMapper;
for (int i = 0; i < availableMetrics.size(); i++) {
metricIndexes.put(availableMetrics.get(i), i);
@ -169,6 +173,18 @@ public class MMappedIndex
return (retVal == null) ? emptySet : retVal;
}
public SmooshedFileMapper getFileMapper()
{
return fileMapper;
}
public void close() throws IOException
{
if (fileMapper != null) {
fileMapper.close();
}
}
public static MMappedIndex fromIndex(Index index)
{
log.info("Converting timestamps");
@ -273,7 +289,8 @@ public class MMappedIndex
dimValueLookups,
dimColumns,
invertedIndexes,
spatialIndexes
spatialIndexes,
null
);
}
}

View File

@ -30,6 +30,7 @@ import com.google.common.collect.Sets;
import com.google.common.primitives.Floats;
import com.metamx.common.ISE;
import com.metamx.druid.input.InputRow;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
import java.util.Arrays;
@ -138,6 +139,12 @@ public class SpatialDimensionRowFormatter
{
return row.getFloatMetric(metric);
}
@Override
public String toString()
{
return row.toString();
}
};
if (!spatialPartialDimNames.isEmpty()) {

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.5.0-SNAPSHOT</version>
<version>0.5.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -57,7 +57,9 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.shard.ShardSpec;
import com.metamx.druid.utils.JodaUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.mapreduce.Job;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -655,22 +657,33 @@ public class HadoopDruidIndexerConfig
return new Path(makeDescriptorInfoDir(), String.format("%s.json", segment.getIdentifier().replace(":", "")));
}
public Path makeSegmentOutputPath(Bucket bucket)
{
final Interval bucketInterval = getGranularitySpec().bucketInterval(bucket.time).get();
return new Path(
String.format(
"%s/%s/%s_%s/%s/%s",
getSegmentOutputDir(),
dataSource,
bucketInterval.getStart().toString(),
bucketInterval.getEnd().toString(),
getVersion(),
bucket.partitionNum
)
);
}
public Path makeSegmentOutputPath(FileSystem fileSystem, Bucket bucket)
{
final Interval bucketInterval = getGranularitySpec().bucketInterval(bucket.time).get();
if (fileSystem instanceof DistributedFileSystem)
{
return new Path(
String.format(
"%s/%s/%s_%s/%s/%s",
getSegmentOutputDir().replace(":", "_"),
dataSource.replace(":", "_"),
bucketInterval.getStart().toString(ISODateTimeFormat.basicDateTime()),
bucketInterval.getEnd().toString(ISODateTimeFormat.basicDateTime()),
getVersion().replace(":", "_"),
bucket.partitionNum
));
}
return new Path(
String.format(
"%s/%s/%s_%s/%s/%s",
getSegmentOutputDir(),
dataSource,
bucketInterval.getStart().toString(),
bucketInterval.getEnd().toString(),
getVersion(),
bucket.partitionNum
));
}
public Job addInputPaths(Job job) throws IOException
{

View File

@ -375,12 +375,14 @@ public class IndexGeneratorJob implements Jobby
Interval interval = config.getGranularitySpec().bucketInterval(bucket.time).get();
int attemptNumber = context.getTaskAttemptID().getId();
Path indexBasePath = config.makeSegmentOutputPath(bucket);
Path indexZipFilePath = new Path(indexBasePath, String.format("index.zip.%s", attemptNumber));
final FileSystem infoFS = config.makeDescriptorInfoDir().getFileSystem(context.getConfiguration());
final FileSystem outputFS = indexBasePath.getFileSystem(context.getConfiguration());
outputFS.mkdirs(indexBasePath);
FileSystem fileSystem = FileSystem.get(context.getConfiguration());
Path indexBasePath = config.makeSegmentOutputPath(fileSystem, bucket);
Path indexZipFilePath = new Path(indexBasePath, String.format("index.zip.%s", attemptNumber));
final FileSystem infoFS = config.makeDescriptorInfoDir().getFileSystem(context.getConfiguration());
final FileSystem outputFS = indexBasePath.getFileSystem(context.getConfiguration());
outputFS.mkdirs(indexBasePath);
Exception caughtException = null;
ZipOutputStream out = null;

View File

@ -27,7 +27,9 @@ import com.metamx.druid.indexer.granularity.UniformGranularitySpec;
import com.metamx.druid.indexer.partitions.PartitionsSpec;
import com.metamx.druid.indexer.updater.DbUpdaterJobSpec;
import com.metamx.druid.jackson.DefaultObjectMapper;
import org.joda.time.Interval;
import org.joda.time.format.ISODateTimeFormat;
import org.junit.Assert;
import org.junit.Test;
@ -429,6 +431,65 @@ public class HadoopDruidIndexerConfigTest
);
}
@Test
public void shouldMakeHDFSCompliantSegmentOutputPath() {
final HadoopDruidIndexerConfig cfg;
try {
cfg = jsonReadWriteRead(
"{"
+ "\"dataSource\": \"the:data:source\","
+ " \"granularitySpec\":{"
+ " \"type\":\"uniform\","
+ " \"gran\":\"hour\","
+ " \"intervals\":[\"2012-07-10/P1D\"]"
+ " },"
+ "\"segmentOutputPath\": \"/tmp/dru:id/data:test\""
+ "}",
HadoopDruidIndexerConfig.class
);
} catch(Exception e) {
throw Throwables.propagate(e);
}
cfg.setVersion("some:brand:new:version");
Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712);
Path path = cfg.makeSegmentOutputPath(new DistributedFileSystem(), bucket);
Assert.assertEquals("/tmp/dru_id/data_test/the_data_source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version/4712", path.toString());
}
@Test
public void shouldMakeDefaultSegmentOutputPathIfNotHDFS() {
final HadoopDruidIndexerConfig cfg;
try {
cfg = jsonReadWriteRead(
"{"
+ "\"dataSource\": \"the:data:source\","
+ " \"granularitySpec\":{"
+ " \"type\":\"uniform\","
+ " \"gran\":\"hour\","
+ " \"intervals\":[\"2012-07-10/P1D\"]"
+ " },"
+ "\"segmentOutputPath\": \"/tmp/dru:id/data:test\""
+ "}",
HadoopDruidIndexerConfig.class
);
} catch(Exception e) {
throw Throwables.propagate(e);
}
cfg.setVersion("some:brand:new:version");
Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712);
Path path = cfg.makeSegmentOutputPath(new LocalFileSystem(), bucket);
Assert.assertEquals("/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:version/4712", path.toString());
}
private <T> T jsonReadWriteRead(String s, Class<T> klass)
{
try {

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.5.0-SNAPSHOT</version>
<version>0.5.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -35,8 +35,13 @@ import com.metamx.druid.loading.S3DataSegmentPuller;
import com.metamx.druid.loading.SegmentLoaderConfig;
import com.metamx.druid.loading.SegmentLoadingException;
import com.metamx.druid.loading.SingleSegmentLoader;
import com.metamx.druid.indexing.common.actions.TaskActionClient;
import com.metamx.druid.indexing.common.actions.TaskActionClientFactory;
import com.metamx.druid.indexing.common.config.TaskConfig;
import com.metamx.druid.indexing.common.task.Task;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.metrics.MonitorScheduler;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import java.io.File;
@ -58,6 +63,7 @@ public class TaskToolbox
private final DataSegmentAnnouncer segmentAnnouncer;
private final ServerView newSegmentServerView;
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
private final MonitorScheduler monitorScheduler;
private final ObjectMapper objectMapper;
public TaskToolbox(
@ -71,6 +77,7 @@ public class TaskToolbox
DataSegmentAnnouncer segmentAnnouncer,
ServerView newSegmentServerView,
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
MonitorScheduler monitorScheduler,
ObjectMapper objectMapper
)
{
@ -84,6 +91,7 @@ public class TaskToolbox
this.segmentAnnouncer = segmentAnnouncer;
this.newSegmentServerView = newSegmentServerView;
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
this.monitorScheduler = monitorScheduler;
this.objectMapper = objectMapper;
}
@ -127,6 +135,11 @@ public class TaskToolbox
return queryRunnerFactoryConglomerate;
}
public MonitorScheduler getMonitorScheduler()
{
return monitorScheduler;
}
public ObjectMapper getObjectMapper()
{
return objectMapper;
@ -143,7 +156,7 @@ public class TaskToolbox
@Override
public File getSegmentLocations()
{
return new File(getTaskWorkDir(), "fetched_segments");
return new File(getTaskWorkDir(), "fetched_segments").toString();
}
}
);
@ -156,7 +169,8 @@ public class TaskToolbox
return retVal;
}
public File getTaskWorkDir() {
public File getTaskWorkDir()
{
return new File(new File(config.getBaseTaskDir(), task.getId()), "work");
}
}

View File

@ -29,6 +29,7 @@ import com.metamx.druid.indexing.common.config.TaskConfig;
import com.metamx.druid.indexing.common.task.Task;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.metrics.MonitorScheduler;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
/**
@ -45,6 +46,7 @@ public class TaskToolboxFactory
private final DataSegmentAnnouncer segmentAnnouncer;
private final ServerView newSegmentServerView;
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
private final MonitorScheduler monitorScheduler;
private final ObjectMapper objectMapper;
public TaskToolboxFactory(
@ -57,6 +59,7 @@ public class TaskToolboxFactory
DataSegmentAnnouncer segmentAnnouncer,
ServerView newSegmentServerView,
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
MonitorScheduler monitorScheduler,
ObjectMapper objectMapper
)
{
@ -69,6 +72,7 @@ public class TaskToolboxFactory
this.segmentAnnouncer = segmentAnnouncer;
this.newSegmentServerView = newSegmentServerView;
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
this.monitorScheduler = monitorScheduler;
this.objectMapper = objectMapper;
}
@ -85,6 +89,7 @@ public class TaskToolboxFactory
segmentAnnouncer,
newSegmentServerView,
queryRunnerFactoryConglomerate,
monitorScheduler,
objectMapper
);
}

View File

@ -65,6 +65,8 @@ public class RemoteTaskActionClient implements TaskActionClient
final String response;
log.info("Submitting action for task[%s] to coordinator[%s]: %s", task.getId(), serviceUri, taskAction);
try {
response = httpClient.post(serviceUri.toURL())
.setContent("application/json", dataToSend)
@ -107,7 +109,7 @@ public class RemoteTaskActionClient implements TaskActionClient
final String scheme;
final String host;
final int port;
final String path = "/mmx/merger/v1/action";
final String path = "/druid/indexer/v1/action";
if (instance == null) {
throw new ISE("Cannot find instance of indexer to talk to!");

View File

@ -127,12 +127,6 @@ public abstract class AbstractTask implements Task
return TaskStatus.running(id);
}
@Override
public void shutdown()
{
// Do nothing.
}
@Override
public String toString()
{

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.io.Closeables;
import com.metamx.common.exception.FormattedException;
@ -30,7 +31,6 @@ import com.metamx.druid.Query;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.coordination.DataSegmentAnnouncer;
import com.metamx.druid.index.v1.IndexGranularity;
import com.metamx.druid.input.InputRow;
import com.metamx.druid.indexing.common.TaskLock;
import com.metamx.druid.indexing.common.TaskStatus;
import com.metamx.druid.indexing.common.TaskToolbox;
@ -38,18 +38,24 @@ import com.metamx.druid.indexing.common.actions.LockAcquireAction;
import com.metamx.druid.indexing.common.actions.LockListAction;
import com.metamx.druid.indexing.common.actions.LockReleaseAction;
import com.metamx.druid.indexing.common.actions.SegmentInsertAction;
import com.metamx.druid.input.InputRow;
import com.metamx.druid.query.FinalizeResultsQueryRunner;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.query.QueryRunnerFactory;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.query.QueryToolChest;
import com.metamx.druid.realtime.FireDepartment;
import com.metamx.druid.realtime.FireDepartmentConfig;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.RealtimeMetricsMonitor;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.realtime.SegmentPublisher;
import com.metamx.druid.realtime.firehose.Firehose;
import com.metamx.druid.realtime.firehose.FirehoseFactory;
import com.metamx.druid.realtime.firehose.GracefulShutdownFirehose;
import com.metamx.druid.realtime.plumber.NoopRejectionPolicyFactory;
import com.metamx.druid.realtime.plumber.Plumber;
import com.metamx.druid.realtime.plumber.RealtimePlumberSchool;
import com.metamx.druid.realtime.plumber.RejectionPolicyFactory;
import com.metamx.druid.realtime.plumber.Sink;
import com.metamx.druid.realtime.plumber.VersioningPolicy;
import com.metamx.emitter.EmittingLogger;
@ -87,20 +93,14 @@ public class RealtimeIndexTask extends AbstractTask
@JsonIgnore
private final IndexGranularity segmentGranularity;
@JsonIgnore
private final RejectionPolicyFactory rejectionPolicyFactory;
@JsonIgnore
private volatile Plumber plumber = null;
@JsonIgnore
private volatile TaskToolbox toolbox = null;
@JsonIgnore
private volatile GracefulShutdownFirehose firehose = null;
@JsonIgnore
private final Object lock = new Object();
@JsonIgnore
private volatile boolean shutdown = false;
private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null;
@JsonCreator
public RealtimeIndexTask(
@ -110,7 +110,8 @@ public class RealtimeIndexTask extends AbstractTask
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig,
@JsonProperty("windowPeriod") Period windowPeriod,
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity,
@JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory
)
{
super(
@ -133,6 +134,7 @@ public class RealtimeIndexTask extends AbstractTask
this.fireDepartmentConfig = fireDepartmentConfig;
this.windowPeriod = windowPeriod;
this.segmentGranularity = segmentGranularity;
this.rejectionPolicyFactory = rejectionPolicyFactory;
}
@Override
@ -151,7 +153,7 @@ public class RealtimeIndexTask extends AbstractTask
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
{
if (plumber != null) {
QueryRunnerFactory<T, Query<T>> factory = toolbox.getQueryRunnerFactoryConglomerate().findFactory(query);
QueryRunnerFactory<T, Query<T>> factory = queryRunnerFactoryConglomerate.findFactory(query);
QueryToolChest<T, Query<T>> toolChest = factory.getToolchest();
return new FinalizeResultsQueryRunner<T>(plumber.getQueryRunner(query), toolChest);
@ -175,21 +177,9 @@ public class RealtimeIndexTask extends AbstractTask
boolean normalExit = true;
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
// Set up firehose
final Period intermediatePersistPeriod = fireDepartmentConfig.getIntermediatePersistPeriod();
synchronized (lock) {
if (shutdown) {
return TaskStatus.success(getId());
}
log.info(
"Wrapping firehose in GracefulShutdownFirehose with segmentGranularity[%s] and windowPeriod[%s]",
segmentGranularity,
windowPeriod
);
firehose = new GracefulShutdownFirehose(firehoseFactory.connect(), segmentGranularity, windowPeriod);
}
final Firehose firehose = firehoseFactory.connect();
// 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.
@ -226,6 +216,28 @@ public class RealtimeIndexTask extends AbstractTask
toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval()));
}
}
@Override
public void announceSegments(Iterable<DataSegment> segments) throws IOException
{
for (DataSegment segment : segments) {
toolbox.getTaskActionClient().submit(new LockAcquireAction(segment.getInterval()));
}
toolbox.getSegmentAnnouncer().announceSegments(segments);
}
@Override
public void unannounceSegments(Iterable<DataSegment> segments) throws IOException
{
try {
toolbox.getSegmentAnnouncer().unannounceSegments(segments);
}
finally {
for (DataSegment segment : segments) {
toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval()));
}
}
}
};
// NOTE: getVersion will block if there is lock contention, which will block plumber.getSink
@ -264,12 +276,22 @@ public class RealtimeIndexTask extends AbstractTask
realtimePlumberSchool.setServerView(toolbox.getNewSegmentServerView());
realtimePlumberSchool.setServiceEmitter(toolbox.getEmitter());
this.toolbox = toolbox;
this.plumber = realtimePlumberSchool.findPlumber(schema, metrics);
if (this.rejectionPolicyFactory != null) {
realtimePlumberSchool.setRejectionPolicyFactory(rejectionPolicyFactory);
}
final FireDepartment fireDepartment = new FireDepartment(schema, fireDepartmentConfig, null, null);
final RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor(ImmutableList.of(fireDepartment));
this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate();
this.plumber = realtimePlumberSchool.findPlumber(schema, fireDepartment.getMetrics());
try {
plumber.startJob();
// Set up metrics emission
toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
// Time to read data!
long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
while (firehose.hasMore()) {
final InputRow inputRow;
@ -281,7 +303,7 @@ public class RealtimeIndexTask extends AbstractTask
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
if (sink == null) {
metrics.incrementThrownAway();
fireDepartment.getMetrics().incrementThrownAway();
log.debug("Throwing away event[%s]", inputRow);
if (System.currentTimeMillis() > nextFlush) {
@ -297,7 +319,7 @@ public class RealtimeIndexTask extends AbstractTask
}
int currCount = sink.add(inputRow);
metrics.incrementProcessed();
fireDepartment.getMetrics().incrementProcessed();
if (currCount >= fireDepartmentConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) {
plumber.persist(firehose.commit());
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
@ -305,7 +327,7 @@ public class RealtimeIndexTask extends AbstractTask
}
catch (FormattedException e) {
log.warn(e, "unparseable line");
metrics.incrementUnparseable();
fireDepartment.getMetrics().incrementUnparseable();
}
}
}
@ -326,6 +348,7 @@ public class RealtimeIndexTask extends AbstractTask
}
finally {
Closeables.closeQuietly(firehose);
toolbox.getMonitorScheduler().removeMonitor(metricsMonitor);
}
}
}
@ -333,22 +356,6 @@ public class RealtimeIndexTask extends AbstractTask
return TaskStatus.success(getId());
}
@Override
public void shutdown()
{
try {
synchronized (lock) {
shutdown = true;
if (firehose != null) {
firehose.shutdown();
}
}
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
@JsonProperty
public Schema getSchema()
{
@ -379,6 +386,12 @@ public class RealtimeIndexTask extends AbstractTask
return segmentGranularity;
}
@JsonProperty("rejectionPolicy")
public RejectionPolicyFactory getRejectionPolicyFactory()
{
return rejectionPolicyFactory;
}
public static class TaskActionSegmentPublisher implements SegmentPublisher
{
final Task task;

View File

@ -134,10 +134,4 @@ public interface Task
* @throws Exception
*/
public TaskStatus run(TaskToolbox toolbox) throws Exception;
/**
* Best-effort task cancellation. May or may not do anything. Calling this multiple times may have
* a stronger effect.
*/
public void shutdown();
}

View File

@ -32,6 +32,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.Closer;
import com.google.common.io.Files;
import com.google.common.io.InputSupplier;
import com.google.common.util.concurrent.ListenableFuture;
@ -48,6 +49,7 @@ import com.metamx.druid.indexing.worker.executor.ExecutorMain;
import com.metamx.emitter.EmittingLogger;
import org.apache.commons.io.FileUtils;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
@ -115,133 +117,135 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
final ProcessHolder processHolder;
try {
if (!attemptDir.mkdirs()) {
throw new IOException(String.format("Could not create directories: %s", attemptDir));
}
final File taskFile = new File(attemptDir, "task.json");
final File statusFile = new File(attemptDir, "status.json");
final File logFile = new File(attemptDir, "log");
// time to adjust process holders
synchronized (tasks) {
final TaskInfo taskInfo = tasks.get(task.getId());
if (taskInfo.shutdown) {
throw new IllegalStateException("Task has been shut down!");
}
if (taskInfo == null) {
throw new ISE("WTF?! TaskInfo disappeared for task: %s", task.getId());
}
if (taskInfo.processHolder != null) {
throw new ISE("WTF?! TaskInfo already has a process holder for task: %s", task.getId());
}
final List<String> command = Lists.newArrayList();
final int childPort = findUnusedPort();
final String childHost = String.format(config.getHostPattern(), childPort);
command.add(config.getJavaCommand());
command.add("-cp");
command.add(config.getJavaClasspath());
Iterables.addAll(
command,
Splitter.on(CharMatcher.WHITESPACE)
.omitEmptyStrings()
.split(config.getJavaOptions())
);
for (String propName : props.stringPropertyNames()) {
command.add(
String.format(
"-D%s=%s",
propName,
props.getProperty(propName)
)
);
}
// Override child JVM specific properties
for (String propName : props.stringPropertyNames()) {
if (propName.startsWith(CHILD_PROPERTY_PREFIX)) {
command.add(
String.format(
"-D%s=%s",
propName.substring(CHILD_PROPERTY_PREFIX.length()),
props.getProperty(propName)
)
);
}
}
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));
command.add(config.getMainClass());
command.add(taskFile.toString());
command.add(statusFile.toString());
jsonMapper.writeValue(taskFile, task);
log.info("Running command: %s", Joiner.on(" ").join(command));
taskInfo.processHolder = new ProcessHolder(
new ProcessBuilder(ImmutableList.copyOf(command)).redirectErrorStream(true).start(),
logFile,
childPort
);
processHolder = taskInfo.processHolder;
}
log.info("Logging task %s output to: %s", task.getId(), logFile);
final OutputStream toProc = processHolder.process.getOutputStream();
final InputStream fromProc = processHolder.process.getInputStream();
final OutputStream toLogfile = Files.newOutputStreamSupplier(logFile).getOutput();
boolean runFailed = false;
final Closer closer = Closer.create();
try {
if (!attemptDir.mkdirs()) {
throw new IOException(String.format("Could not create directories: %s", attemptDir));
}
final File taskFile = new File(attemptDir, "task.json");
final File statusFile = new File(attemptDir, "status.json");
final File logFile = new File(attemptDir, "log");
// time to adjust process holders
synchronized (tasks) {
final TaskInfo taskInfo = tasks.get(task.getId());
if (taskInfo.shutdown) {
throw new IllegalStateException("Task has been shut down!");
}
if (taskInfo == null) {
throw new ISE("WTF?! TaskInfo disappeared for task: %s", task.getId());
}
if (taskInfo.processHolder != null) {
throw new ISE("WTF?! TaskInfo already has a process holder for task: %s", task.getId());
}
final List<String> command = Lists.newArrayList();
final int childPort = findUnusedPort();
final String childHost = String.format(config.getHostPattern(), childPort);
command.add(config.getJavaCommand());
command.add("-cp");
command.add(config.getJavaClasspath());
Iterables.addAll(
command,
Splitter.on(CharMatcher.WHITESPACE)
.omitEmptyStrings()
.split(config.getJavaOptions())
);
for (String propName : props.stringPropertyNames()) {
for (String allowedPrefix : config.getAllowedPrefixes()) {
if (propName.startsWith(allowedPrefix)) {
command.add(
String.format(
"-D%s=%s",
propName,
props.getProperty(propName)
)
);
}
}
}
// Override child JVM specific properties
for (String propName : props.stringPropertyNames()) {
if (propName.startsWith(CHILD_PROPERTY_PREFIX)) {
command.add(
String.format(
"-D%s=%s",
propName.substring(CHILD_PROPERTY_PREFIX.length()),
props.getProperty(propName)
)
);
}
}
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));
command.add(config.getMainClass());
command.add(taskFile.toString());
command.add(statusFile.toString());
jsonMapper.writeValue(taskFile, task);
log.info("Running command: %s", Joiner.on(" ").join(command));
taskInfo.processHolder = new ProcessHolder(
new ProcessBuilder(ImmutableList.copyOf(command)).redirectErrorStream(true).start(),
logFile,
childPort
);
processHolder = taskInfo.processHolder;
processHolder.registerWithCloser(closer);
}
log.info("Logging task %s output to: %s", task.getId(), logFile);
final InputStream fromProc = processHolder.process.getInputStream();
final OutputStream toLogfile = closer.register(
Files.newOutputStreamSupplier(logFile).getOutput()
);
boolean runFailed = true;
ByteStreams.copy(fromProc, toLogfile);
final int statusCode = processHolder.process.waitFor();
log.info("Process exited with status[%d] for task: %s", statusCode, task.getId());
if (statusCode != 0) {
runFailed = true;
if (statusCode == 0) {
runFailed = false;
}
}
catch (Exception e) {
log.warn(e, "Failed to read from process for task: %s", task.getId());
runFailed = true;
}
finally {
Closeables.closeQuietly(fromProc);
Closeables.closeQuietly(toLogfile);
Closeables.closeQuietly(toProc);
}
// Upload task logs
// Upload task logs
// XXX: Consider uploading periodically for very long-lived tasks to prevent
// XXX: bottlenecks at the end or the possibility of losing a lot of logs all
// XXX: at once.
// XXX: Consider uploading periodically for very long-lived tasks to prevent
// XXX: bottlenecks at the end or the possibility of losing a lot of logs all
// XXX: at once.
taskLogPusher.pushTaskLog(task.getId(), logFile);
taskLogPusher.pushTaskLog(task.getId(), logFile);
if (!runFailed) {
// Process exited successfully
return jsonMapper.readValue(statusFile, TaskStatus.class);
} else {
// Process exited unsuccessfully
return TaskStatus.failure(task.getId());
if (!runFailed) {
// Process exited successfully
return jsonMapper.readValue(statusFile, TaskStatus.class);
} else {
// Process exited unsuccessfully
return TaskStatus.failure(task.getId());
}
} catch (Throwable t) {
throw closer.rethrow(t);
} finally {
closer.close();
}
}
catch (Exception e) {
@ -307,31 +311,9 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
}
if (taskInfo.processHolder != null) {
final int shutdowns = taskInfo.processHolder.shutdowns.getAndIncrement();
if (shutdowns == 0) {
log.info("Attempting to gracefully shutdown task: %s", taskid);
try {
// This is gross, but it may still be nicer than talking to the forked JVM via HTTP.
final OutputStream out = taskInfo.processHolder.process.getOutputStream();
out.write(
jsonMapper.writeValueAsBytes(
ImmutableMap.of(
"shutdown",
"now"
)
)
);
out.write('\n');
out.flush();
}
catch (IOException e) {
throw Throwables.propagate(e);
}
} else {
// Will trigger normal failure mechanisms due to process exit
log.info("Killing process for task: %s", taskid);
taskInfo.processHolder.process.destroy();
}
// Will trigger normal failure mechanisms due to process exit
log.info("Killing process for task: %s", taskid);
taskInfo.processHolder.process.destroy();
}
}
@ -425,7 +407,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
private final Process process;
private final File logFile;
private final int port;
private final AtomicInteger shutdowns = new AtomicInteger(0);
private ProcessHolder(Process process, File logFile, int port)
{
@ -433,5 +414,11 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
this.logFile = logFile;
this.port = port;
}
private void registerWithCloser(Closer closer)
{
closer.register(process.getInputStream());
closer.register(process.getOutputStream());
}
}
}

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