Merge branch 'master' into subquery

This commit is contained in:
Yuval Oren 2014-01-24 14:42:41 -08:00
commit cd719eab3c
166 changed files with 7538 additions and 2786 deletions

View File

@ -30,4 +30,4 @@ echo "For examples, see: "
echo " "
ls -1 examples/*/*sh
echo " "
echo "See also http://druid.io/docs/0.6.51"
echo "See also http://druid.io/docs/0.6.52"

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -21,10 +21,15 @@ package io.druid.concurrent;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.RejectedExecutionHandler;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
/**
*/
@ -49,4 +54,31 @@ public class Execs
{
return new ThreadFactoryBuilder().setDaemon(true).setNameFormat(nameFormat).build();
}
/**
* @param nameFormat nameformat for threadFactory
* @param capacity maximum capacity after which the executorService will block on accepting new tasks
* @return ExecutorService which blocks accepting new tasks when the capacity reached
*/
public static ExecutorService newBlockingSingleThreaded(String nameFormat, int capacity)
{
return new ThreadPoolExecutor(
1, 1,
0L, TimeUnit.MILLISECONDS,
new ArrayBlockingQueue<Runnable>(capacity), makeThreadFactory(nameFormat)
, new RejectedExecutionHandler()
{
@Override
public void rejectedExecution(Runnable r, ThreadPoolExecutor executor)
{
try {
((ArrayBlockingQueue) executor.getQueue()).put(r);
}
catch (InterruptedException e) {
throw new RejectedExecutionException("Got Interrupted while adding to the Queue");
}
}
}
);
}
}

View File

@ -0,0 +1,92 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.concurrent;
import com.google.common.base.Throwables;
import org.junit.Assert;
import org.junit.Test;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
public class ExecsTest
{
@Test
public void testBlockingExecutorService() throws Exception
{
final int capacity = 3;
final ExecutorService blockingExecutor = Execs.newBlockingSingleThreaded("test%d", capacity);
final CountDownLatch queueFullSignal = new CountDownLatch(capacity + 1);
final CountDownLatch taskCompletedSignal = new CountDownLatch(2 * capacity);
final CountDownLatch taskStartSignal = new CountDownLatch(1);
final AtomicInteger producedCount = new AtomicInteger();
final AtomicInteger consumedCount = new AtomicInteger();
ExecutorService producer = Executors.newSingleThreadExecutor();
producer.submit(
new Runnable()
{
public void run()
{
for (int i = 0; i < 2 * capacity; i++) {
final int taskID = i;
System.out.println("Produced task" + taskID);
blockingExecutor.submit(
new Runnable()
{
@Override
public void run()
{
System.out.println("Starting task" + taskID);
try {
taskStartSignal.await();
consumedCount.incrementAndGet();
taskCompletedSignal.countDown();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
System.out.println("Completed task" + taskID);
}
}
);
producedCount.incrementAndGet();
queueFullSignal.countDown();
}
}
}
);
queueFullSignal.await();
// verify that the producer blocks
Assert.assertEquals(capacity + 1, producedCount.get());
// let the tasks run
taskStartSignal.countDown();
// wait until all tasks complete
taskCompletedSignal.await();
// verify all tasks consumed
Assert.assertEquals(2 * capacity, consumedCount.get());
// cleanup
blockingExecutor.shutdown();
producer.shutdown();
}
}

View File

@ -37,7 +37,7 @@ The broker module uses several of the default modules in [Configuration](Configu
|Property|Possible Values|Description|Default|
|--------|---------------|-----------|-------|
|`druid.broker.cache.type`|`local`, `memcache`|The type of cache to use for queries.|`local`|
|`druid.broker.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`|
|`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to compute nodes. `random` choose randomly, `connectionCount` picks the node with the fewest number of active connections to|`random`|
#### Local Cache

View File

@ -19,13 +19,13 @@ Clone Druid and build it:
git clone https://github.com/metamx/druid.git druid
cd druid
git fetch --tags
git checkout druid-0.6.51
git checkout druid-0.6.52
./build.sh
```
### Downloading the DSK (Druid Standalone Kit)
[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.51-bin.tar.gz) a stand-alone tarball and run it:
[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.52-bin.tar.gz) a stand-alone tarball and run it:
``` bash
tar -xzf druid-services-0.X.X-bin.tar.gz

View File

@ -97,7 +97,6 @@ This describes the data schema for the output Druid segment. More information ab
|aggregators|Array of Objects|The list of aggregators to use to aggregate colliding rows together.|yes|
|dataSource|String|The name of the dataSource that the segment belongs to.|yes|
|indexGranularity|String|The granularity of the data inside the segment. E.g. a value of "minute" will mean that data is aggregated at minutely granularity. That is, if there are collisions in the tuple (minute(timestamp), dimensions), then it will aggregate values together using the aggregators instead of storing individual rows.|yes|
|segmentGranularity|String|The granularity of the segment as a whole. This is generally larger than the index granularity and describes the rate at which the realtime server will push segments out for historical servers to take over.|yes|
|shardSpec|Object|This describes the shard that is represented by this server. This must be specified properly in order to have multiple realtime nodes indexing the same data stream in a sharded fashion.|no|
### Config

View File

@ -27,7 +27,7 @@ druid.host=localhost
druid.service=realtime
druid.port=8083
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.51"]
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.52"]
druid.zk.service.host=localhost

View File

@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu
### Download a Tarball
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.51-bin.tar.gz). Download this file to a directory of your choosing.
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.52-bin.tar.gz). Download this file to a directory of your choosing.
You can extract the awesomeness within by issuing:
@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz
Not too lost so far right? That's great! If you cd into the directory:
```
cd druid-services-0.6.51
cd druid-services-0.6.52
```
You should see a bunch of files:

View File

@ -219,9 +219,9 @@ Congratulations! The segment has completed building. Once a segment is built, a
You should see the following logs on the coordinator:
```bash
2013-10-09 21:41:54,368 INFO [Coordinator-Exec--0] io.druid.server.coordinator.DruidCoordinatorLogger - [_default_tier] : Assigned 1 segments among 1 servers
2013-10-09 21:41:54,369 INFO [Coordinator-Exec--0] io.druid.server.coordinator.DruidCoordinatorLogger - Load Queues:
2013-10-09 21:41:54,369 INFO [Coordinator-Exec--0] io.druid.server.coordinator.DruidCoordinatorLogger - Server[localhost:8081, historical, _default_tier] has 1 left to load, 0 left to drop, 4,477 bytes queued, 4,477 bytes served.
2013-10-09 21:41:54,368 INFO [Coordinator-Exec--0] io.druid.server.coordinator.helper.DruidCoordinatorLogger - [_default_tier] : Assigned 1 segments among 1 servers
2013-10-09 21:41:54,369 INFO [Coordinator-Exec--0] io.druid.server.coordinator.helper.DruidCoordinatorLogger - Load Queues:
2013-10-09 21:41:54,369 INFO [Coordinator-Exec--0] io.druid.server.coordinator.helper.DruidCoordinatorLogger - Server[localhost:8081, historical, _default_tier] has 1 left to load, 0 left to drop, 4,477 bytes queued, 4,477 bytes served.
```
These logs indicate that the coordinator has assigned our new segment to the historical node to download and serve. If you look at the historical node logs, you should see:

View File

@ -45,7 +45,7 @@ With real-world data, we recommend having a message bus such as [Apache Kafka](h
<a id="set-up-kafka"></a>
#### Setting up Kafka
[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.51/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node.
[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.52/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node.
Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html).

View File

@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes as well as and exter
If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first.
You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.51-bin.tar.gz)
You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.52-bin.tar.gz)
and untar the contents within by issuing:
@ -149,7 +149,7 @@ druid.port=8081
druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.51"]
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.52"]
# Dummy read only AWS account (used to download example data)
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
@ -240,7 +240,7 @@ druid.port=8083
druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.51","io.druid.extensions:druid-kafka-seven:0.6.51"]
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.52","io.druid.extensions:druid-kafka-seven:0.6.52"]
# Change this config to db to hand off to the rest of the Druid cluster
druid.publish.type=noop

View File

@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu
h3. Download a Tarball
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.51-bin.tar.gz)
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.52-bin.tar.gz)
Download this file to a directory of your choosing.
You can extract the awesomeness within by issuing:
@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz
Not too lost so far right? That's great! If you cd into the directory:
```
cd druid-services-0.6.51
cd druid-services-0.6.52
```
You should see a bunch of files:

View File

@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source.
h3. Download a Tarball
We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.51-bin.tar.gz.
We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.52-bin.tar.gz.
Download this bad boy to a directory of your choosing.
You can extract the awesomeness within by issuing:

View File

@ -4,7 +4,7 @@ druid.port=8081
druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.51"]
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.52"]
# Dummy read only AWS account (used to download example data)
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b

View File

@ -4,7 +4,7 @@ druid.port=8083
druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.51","io.druid.extensions:druid-kafka-seven:0.6.51","io.druid.extensions:druid-rabbitmq:0.6.51"]
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.52","io.druid.extensions:druid-kafka-seven:0.6.52","io.druid.extensions:druid-rabbitmq:0.6.52"]
# Change this config to db to hand off to the rest of the Druid cluster
druid.publish.type=noop

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

82
hll/pom.xml Normal file
View File

@ -0,0 +1,82 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ Druid - a distributed column store.
~ Copyright (C) 2012, 2013 Metamarkets Group Inc.
~
~ This program is free software; you can redistribute it and/or
~ modify it under the terms of the GNU General Public License
~ as published by the Free Software Foundation; either version 2
~ of the License, or (at your option) any later version.
~
~ This program is distributed in the hope that it will be useful,
~ but WITHOUT ANY WARRANTY; without even the implied warranty of
~ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
~ GNU General Public License for more details.
~
~ You should have received a copy of the GNU General Public License
~ along with this program; if not, write to the Free Software
~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
-->
<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>io.druid.extensions</groupId>
<artifactId>druid-hll</artifactId>
<name>druid-hll</name>
<description>druid-hll</description>
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-api</artifactId>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>com.metamx</groupId>
<artifactId>emitter</artifactId>
</dependency>
<dependency>
<groupId>net.sf.trove4j</groupId>
<artifactId>trove4j</artifactId>
<version>3.0.3</version>
</dependency>
<dependency>
<groupId>commons-codec</groupId>
<artifactId>commons-codec</artifactId>
</dependency>
<!-- Tests -->
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<archive>
<manifest>
<addDefaultImplementationEntries>true</addDefaultImplementationEntries>
<addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
</manifest>
</archive>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,137 @@
/*
* 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 io.druid.query.aggregation;
import com.google.common.hash.Hashing;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import gnu.trove.map.TIntByteMap;
import gnu.trove.map.hash.TIntByteHashMap;
import io.druid.segment.ObjectColumnSelector;
import java.util.Comparator;
public class HyperloglogAggregator implements Aggregator
{
private static final Logger log = new Logger(HyperloglogAggregator.class);
public static final int log2m = 12;
public static final int m = (int) Math.pow(2, log2m);
public static final double alphaMM = (0.7213 / (1 + 1.079 / m)) * m * m;
private final String name;
private final ObjectColumnSelector selector;
private TIntByteHashMap ibMap;
static final Comparator COMPARATOR = new Comparator()
{
@Override
public int compare(Object o, Object o1)
{
return o.equals(o1) ? 0 : 1;
}
};
public static Object combine(Object lhs, Object rhs)
{
final TIntByteMap newIbMap = new TIntByteHashMap((TIntByteMap) lhs);
final TIntByteMap rightIbMap = (TIntByteMap) rhs;
final int[] keys = rightIbMap.keys();
for (int key : keys) {
if (newIbMap.get(key) == newIbMap.getNoEntryValue() || rightIbMap.get(key) > newIbMap.get(key)) {
newIbMap.put(key, rightIbMap.get(key));
}
}
return newIbMap;
}
public HyperloglogAggregator(String name, ObjectColumnSelector selector)
{
this.name = name;
this.selector = selector;
this.ibMap = new TIntByteHashMap();
}
@Override
public void aggregate()
{
final Object value = selector.get();
if (value == null) {
return;
}
if (value instanceof TIntByteHashMap) {
final TIntByteHashMap newIbMap = (TIntByteHashMap) value;
final int[] indexes = newIbMap.keys();
for (int index : indexes) {
if (ibMap.get(index) == ibMap.getNoEntryValue() || newIbMap.get(index) > ibMap.get(index)) {
ibMap.put(index, newIbMap.get(index));
}
}
} else if (value instanceof String) {
log.debug("value [%s]", selector.get());
final long id = Hashing.murmur3_128().hashString((String) (value)).asLong();
final int bucket = (int) (id >>> (Long.SIZE - log2m));
final int zerolength = Long.numberOfLeadingZeros((id << log2m) | (1 << (log2m - 1)) + 1) + 1;
if (ibMap.get(bucket) == ibMap.getNoEntryValue() || ibMap.get(bucket) < (byte) zerolength) {
ibMap.put(bucket, (byte) zerolength);
}
} else {
throw new ISE("Aggregate does not support values of type[%s]", value.getClass().getName());
}
}
@Override
public void reset()
{
this.ibMap = new TIntByteHashMap();
}
@Override
public Object get()
{
return ibMap;
}
@Override
public float getFloat()
{
throw new UnsupportedOperationException("HyperloglogAggregator does not support getFloat()");
}
@Override
public String getName()
{
return name;
}
@Override
public void close()
{
// do nothing
}
}

View File

@ -0,0 +1,209 @@
/*
* 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 io.druid.query.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.metamx.common.logger.Logger;
import gnu.trove.map.hash.TIntByteHashMap;
import io.druid.segment.ColumnSelectorFactory;
import org.apache.commons.codec.binary.Base64;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
public class HyperloglogAggregatorFactory implements AggregatorFactory
{
private static final Logger log = new Logger(HyperloglogAggregatorFactory.class);
private static final byte[] CACHE_KEY = new byte[]{0x37};
private final String name;
private final String fieldName;
@JsonCreator
public HyperloglogAggregatorFactory(
@JsonProperty("name") final String name,
@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;
}
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
return new HyperloglogAggregator(
name,
metricFactory.makeObjectColumnSelector(fieldName)
);
}
@Override
public BufferAggregator factorizeBuffered(
ColumnSelectorFactory metricFactory
)
{
return new HyperloglogBufferAggregator(
metricFactory.makeObjectColumnSelector(fieldName)
);
}
@Override
public Comparator getComparator()
{
return HyperloglogAggregator.COMPARATOR;
}
@Override
public Object combine(Object lhs, Object rhs)
{
if (rhs == null) {
return lhs;
}
if (lhs == null) {
return rhs;
}
return HyperloglogAggregator.combine(lhs, rhs);
}
@Override
public AggregatorFactory getCombiningFactory()
{
log.debug("factory name: %s", name);
return new HyperloglogAggregatorFactory(name, fieldName);
}
@Override
public Object deserialize(Object object)
{
log.debug("class name: [%s]:value [%s]", object.getClass().getName(), object);
final String k = (String) object;
final byte[] ibmapByte = Base64.decodeBase64(k);
final ByteBuffer buffer = ByteBuffer.wrap(ibmapByte);
final int keylength = buffer.getInt();
final int valuelength = buffer.getInt();
TIntByteHashMap newIbMap;
if (keylength == 0) {
newIbMap = new TIntByteHashMap();
} else {
final int[] keys = new int[keylength];
final byte[] values = new byte[valuelength];
for (int i = 0; i < keylength; i++) {
keys[i] = buffer.getInt();
}
buffer.get(values);
newIbMap = new TIntByteHashMap(keys, values);
}
return newIbMap;
}
@Override
public Object finalizeComputation(Object object)
{
final TIntByteHashMap ibMap = (TIntByteHashMap) object;
final int[] keys = ibMap.keys();
final int count = keys.length;
double registerSum = 0;
double zeros = 0.0;
for (int key : keys) {
int val = ibMap.get(key);
registerSum += 1.0 / (1 << val);
if (val == 0) {
zeros++;
}
}
registerSum += (HyperloglogAggregator.m - count);
zeros += HyperloglogAggregator.m - count;
double estimate = HyperloglogAggregator.alphaMM * (1.0 / registerSum);
if (estimate <= (5.0 / 2.0) * (HyperloglogAggregator.m)) {
// Small Range Estimate
return Math.round(HyperloglogAggregator.m * Math.log(HyperloglogAggregator.m / zeros));
} else {
return Math.round(estimate);
}
}
@JsonProperty
public String getFieldName()
{
return fieldName;
}
@Override
@JsonProperty
public String getName()
{
return name;
}
@Override
public List<String> requiredFields()
{
return Arrays.asList(fieldName);
}
@Override
public byte[] getCacheKey()
{
byte[] fieldNameBytes = fieldName.getBytes();
return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_KEY)
.put(fieldNameBytes).array();
}
@Override
public String getTypeName()
{
return "hyperloglog";
}
@Override
public int getMaxIntermediateSize()
{
return HyperloglogAggregator.m;
}
@Override
public Object getAggregatorStartValue()
{
return new TIntByteHashMap();
}
}

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 io.druid.query.aggregation;
import gnu.trove.map.hash.TIntByteHashMap;
import gnu.trove.procedure.TIntByteProcedure;
import io.druid.segment.ObjectColumnSelector;
import java.nio.ByteBuffer;
public class HyperloglogBufferAggregator implements BufferAggregator
{
private final ObjectColumnSelector selector;
public HyperloglogBufferAggregator(ObjectColumnSelector selector)
{
this.selector = selector;
}
/*
* byte 1 key length byte 2 value length byte 3...n key array byte n+1....
* value array
*/
@Override
public void init(ByteBuffer buf, int position)
{
for (int i = 0; i < HyperloglogAggregator.m; i++) {
buf.put(position + i, (byte) 0);
}
}
@Override
public void aggregate(ByteBuffer buf, int position)
{
final ByteBuffer fb = buf;
final int fp = position;
final TIntByteHashMap newObj = (TIntByteHashMap) (selector.get());
newObj.forEachEntry(
new TIntByteProcedure()
{
public boolean execute(int a, byte b)
{
if (b > fb.get(fp + a)) {
fb.put(fp + a, b);
}
return true;
}
}
);
}
@Override
public Object get(ByteBuffer buf, int position)
{
final TIntByteHashMap ret = new TIntByteHashMap();
for (int i = 0; i < HyperloglogAggregator.m; i++) {
if (buf.get(position + i) != 0) {
ret.put(i, buf.get(position + i));
}
}
return ret;
}
@Override
public float getFloat(ByteBuffer buf, int position)
{
throw new UnsupportedOperationException("HyperloglogAggregator does not support getFloat()");
}
@Override
public void close()
{
// do nothing
}
}

View File

@ -0,0 +1,137 @@
/*
* 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 io.druid.query.aggregation;
import gnu.trove.map.hash.TIntByteHashMap;
import io.druid.data.input.InputRow;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.ObjectStrategy;
import io.druid.segment.serde.ColumnPartSerde;
import io.druid.segment.serde.ComplexColumnPartSerde;
import io.druid.segment.serde.ComplexColumnPartSupplier;
import io.druid.segment.serde.ComplexMetricExtractor;
import io.druid.segment.serde.ComplexMetricSerde;
import java.nio.ByteBuffer;
import java.util.List;
public class HyperloglogComplexMetricSerde extends ComplexMetricSerde
{
@Override
public String getTypeName()
{
return "hyperloglog";
}
@Override
public ComplexMetricExtractor getExtractor()
{
return new HyperloglogComplexMetricExtractor();
}
@Override
public ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder)
{
GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy());
builder.setType(ValueType.COMPLEX);
builder.setComplexColumn(new ComplexColumnPartSupplier("hyperloglog", column));
return new ComplexColumnPartSerde(column, "hyperloglog");
}
@Override
public ObjectStrategy getObjectStrategy()
{
return new HyperloglogObjectStrategy();
}
public static class HyperloglogObjectStrategy implements ObjectStrategy<TIntByteHashMap>
{
@Override
public Class<? extends TIntByteHashMap> getClazz()
{
return TIntByteHashMap.class;
}
@Override
public TIntByteHashMap fromByteBuffer(ByteBuffer buffer, int numBytes)
{
int keylength = buffer.getInt();
int valuelength = buffer.getInt();
if (keylength == 0) {
return new TIntByteHashMap();
}
int[] keys = new int[keylength];
byte[] values = new byte[valuelength];
for (int i = 0; i < keylength; i++) {
keys[i] = buffer.getInt();
}
buffer.get(values);
TIntByteHashMap tib = new TIntByteHashMap(keys, values);
return tib;
}
@Override
public byte[] toBytes(TIntByteHashMap val)
{
TIntByteHashMap ibmap = val;
int[] indexesResult = ibmap.keys();
byte[] valueResult = ibmap.values();
ByteBuffer buffer = ByteBuffer.allocate(4 * indexesResult.length + valueResult.length + 8);
byte[] result = new byte[4 * indexesResult.length + valueResult.length + 8];
buffer.putInt((int) indexesResult.length);
buffer.putInt((int) valueResult.length);
for (int i = 0; i < indexesResult.length; i++) {
buffer.putInt(indexesResult[i]);
}
buffer.put(valueResult);
buffer.flip();
buffer.get(result);
return result;
}
@Override
public int compare(TIntByteHashMap o1, TIntByteHashMap o2)
{
return o1.equals(o2) ? 0 : 1;
}
}
public static class HyperloglogComplexMetricExtractor implements ComplexMetricExtractor
{
@Override
public Class<?> extractedClass()
{
return List.class;
}
@Override
public Object extractValue(InputRow inputRow, String metricName)
{
return inputRow.getRaw(metricName);
}
}
}

View File

@ -0,0 +1,140 @@
/*
* 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 io.druid.query.aggregation;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.JsonDeserializer;
import com.fasterxml.jackson.databind.JsonSerializer;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import gnu.trove.map.hash.TIntByteHashMap;
import io.druid.initialization.DruidModule;
import io.druid.segment.serde.ComplexMetrics;
import org.apache.commons.codec.binary.Base64;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
/**
*/
public class HyperloglogDruidModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(
new HyperloglogJacksonSerdeModule().registerSubtypes(
new NamedType(HyperloglogAggregatorFactory.class, "hyperloglog")
)
);
}
@Override
public void configure(Binder binder)
{
if (ComplexMetrics.getSerdeForType("hyperloglog") == null) {
ComplexMetrics.registerSerde("hyperloglog", new HyperloglogComplexMetricSerde());
}
}
public static class HyperloglogJacksonSerdeModule extends SimpleModule
{
public HyperloglogJacksonSerdeModule()
{
super("Hyperloglog deserializers");
addDeserializer(
TIntByteHashMap.class,
new JsonDeserializer<TIntByteHashMap>()
{
@Override
public TIntByteHashMap deserialize(
JsonParser jp,
DeserializationContext ctxt
) throws IOException
{
byte[] ibmapByte = Base64.decodeBase64(jp.getText());
ByteBuffer buffer = ByteBuffer.wrap(ibmapByte);
int keylength = buffer.getInt();
int valuelength = buffer.getInt();
if (keylength == 0) {
return (new TIntByteHashMap());
}
int[] keys = new int[keylength];
byte[] values = new byte[valuelength];
for (int i = 0; i < keylength; i++) {
keys[i] = buffer.getInt();
}
buffer.get(values);
return (new TIntByteHashMap(keys, values));
}
}
);
addSerializer(
TIntByteHashMap.class,
new JsonSerializer<TIntByteHashMap>()
{
@Override
public void serialize(
TIntByteHashMap ibmap,
JsonGenerator jsonGenerator,
SerializerProvider serializerProvider
)
throws IOException, JsonProcessingException
{
int[] indexesResult = ibmap.keys();
byte[] valueResult = ibmap.values();
ByteBuffer buffer = ByteBuffer
.allocate(
4 * indexesResult.length
+ valueResult.length + 8
);
byte[] result = new byte[4 * indexesResult.length
+ valueResult.length + 8];
buffer.putInt((int) indexesResult.length);
buffer.putInt((int) valueResult.length);
for (int i = 0; i < indexesResult.length; i++) {
buffer.putInt(indexesResult[i]);
}
buffer.put(valueResult);
buffer.flip();
buffer.get(result);
String str = Base64.encodeBase64String(result);
jsonGenerator.writeString(str);
}
}
);
}
}
}

View File

@ -0,0 +1 @@
io.druid.query.aggregation.HyperloglogDruidModule

View File

@ -0,0 +1,162 @@
/*
* 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 io.druid.query.aggregation;
import gnu.trove.map.hash.TIntByteHashMap;
import org.junit.Assert;
import org.junit.Test;
import java.util.Comparator;
public class HyperloglogAggregatorTest
{
@Test
public void testAggregate()
{
final TestHllComplexMetricSelector selector = new TestHllComplexMetricSelector();
final HyperloglogAggregatorFactory aggFactory = new HyperloglogAggregatorFactory("billy", "billyG");
final HyperloglogAggregator agg = new HyperloglogAggregator("billy", selector);
Assert.assertEquals("billy", agg.getName());
Assert.assertEquals(0L, aggFactory.finalizeComputation(agg.get()));
Assert.assertEquals(0L, aggFactory.finalizeComputation(agg.get()));
Assert.assertEquals(0L, aggFactory.finalizeComputation(agg.get()));
aggregate(selector, agg);
aggregate(selector, agg);
aggregate(selector, agg);
Assert.assertEquals(3L, aggFactory.finalizeComputation(agg.get()));
Assert.assertEquals(3L, aggFactory.finalizeComputation(agg.get()));
Assert.assertEquals(3L, aggFactory.finalizeComputation(agg.get()));
aggregate(selector, agg);
aggregate(selector, agg);
Assert.assertEquals(5L, aggFactory.finalizeComputation(agg.get()));
Assert.assertEquals(5L, aggFactory.finalizeComputation(agg.get()));
}
@Test
public void testComparator()
{
final TestHllComplexMetricSelector selector = new TestHllComplexMetricSelector();
final Comparator comp = new HyperloglogAggregatorFactory("billy", "billyG").getComparator();
final HyperloglogAggregator agg = new HyperloglogAggregator("billy", selector);
Object first = new TIntByteHashMap((TIntByteHashMap) agg.get());
agg.aggregate();
Assert.assertEquals(0, comp.compare(first, first));
Assert.assertEquals(0, comp.compare(agg.get(), agg.get()));
Assert.assertEquals(1, comp.compare(agg.get(), first));
}
@Test
public void testHighCardinalityAggregate()
{
final TestHllComplexMetricSelector selector = new TestHllComplexMetricSelector();
final HyperloglogAggregatorFactory aggFactory = new HyperloglogAggregatorFactory("billy", "billyG");
final HyperloglogAggregator agg = new HyperloglogAggregator("billy", selector);
final int card = 100000;
for (int i = 0; i < card; i++) {
aggregate(selector, agg);
}
Assert.assertEquals(99443L, aggFactory.finalizeComputation(agg.get()));
}
// Provides a nice printout of error rates as a function of cardinality
//@Test
public void benchmarkAggregation() throws Exception
{
final TestHllComplexMetricSelector selector = new TestHllComplexMetricSelector();
final HyperloglogAggregatorFactory aggFactory = new HyperloglogAggregatorFactory("billy", "billyG");
double error = 0.0d;
int count = 0;
final int[] valsToCheck = {
10, 20, 50, 100, 1000, 2000, 5000, 10000, 20000, 50000, 100000, 1000000, 2000000, 10000000, Integer.MAX_VALUE
};
for (int numThings : valsToCheck) {
long startTime = System.currentTimeMillis();
final HyperloglogAggregator agg = new HyperloglogAggregator("billy", selector);
for (int i = 0; i < numThings; ++i) {
if (i != 0 && i % 100000000 == 0) {
++count;
error = computeError(error, count, i, (Long) aggFactory.finalizeComputation(agg.get()), startTime);
}
aggregate(selector, agg);
}
++count;
error = computeError(error, count, numThings, (Long) aggFactory.finalizeComputation(agg.get()), startTime);
}
}
//@Test
public void benchmarkCombine() throws Exception
{
int count;
long totalTime = 0;
final TestHllComplexMetricSelector selector = new TestHllComplexMetricSelector();
TIntByteHashMap combined = new TIntByteHashMap();
for (count = 0; count < 1000000; ++count) {
final HyperloglogAggregator agg = new HyperloglogAggregator("billy", selector);
aggregate(selector, agg);
long start = System.nanoTime();
combined = (TIntByteHashMap) HyperloglogAggregator.combine(agg.get(), combined);
totalTime += System.nanoTime() - start;
}
System.out.printf("benchmarkCombine took %d ms%n", totalTime / 1000000);
}
private double computeError(double error, int count, long exactValue, long estimatedValue, long startTime)
{
final double errorThisTime = Math.abs((double) exactValue - estimatedValue) / exactValue;
error += errorThisTime;
System.out.printf(
"%,d ==? %,d in %,d millis. actual error[%,f%%], avg. error [%,f%%]%n",
exactValue,
estimatedValue,
System.currentTimeMillis() - startTime,
100 * errorThisTime,
(error / count) * 100
);
return error;
}
private void aggregate(TestHllComplexMetricSelector selector, HyperloglogAggregator agg)
{
agg.aggregate();
selector.increment();
}
}

View File

@ -0,0 +1,45 @@
/*
* 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 io.druid.query.aggregation;
import io.druid.segment.ObjectColumnSelector;
public class TestHllComplexMetricSelector implements ObjectColumnSelector<String>
{
private int index = 0;
@Override
public Class<String> classOfObject()
{
return String.class;
}
@Override
public String get()
{
return String.valueOf(index);
}
public void increment()
{
++index;
}
}

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -19,11 +19,9 @@
package io.druid.indexer;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.logger.Logger;
import io.druid.db.DbConnector;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.skife.jdbi.v2.Handle;
@ -39,8 +37,6 @@ public class DbUpdaterJob implements Jobby
{
private static final Logger log = new Logger(DbUpdaterJob.class);
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
private final HadoopDruidIndexerConfig config;
private final IDBI dbi;
@ -82,7 +78,7 @@ public class DbUpdaterJob implements Jobby
.put("partitioned", segment.getShardSpec().getPartitionNum())
.put("version", segment.getVersion())
.put("used", true)
.put("payload", jsonMapper.writeValueAsString(segment))
.put("payload", HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(segment))
.build()
);

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -194,7 +194,7 @@ public class RealtimeIndexTask extends AbstractTask
final RealtimePlumberSchool realtimePlumberSchool = new RealtimePlumberSchool(
windowPeriod,
new File(toolbox.getTaskWorkDir(), "persist"),
segmentGranularity
segmentGranularity, fireDepartmentConfig.getMaxPendingPersists()
);
final SegmentPublisher segmentPublisher = new TaskActionSegmentPublisher(this, toolbox);

View File

@ -30,6 +30,8 @@ public interface AutoScalingStrategy
public AutoScalingData terminate(List<String> ips);
public AutoScalingData terminateWithIds(List<String> ids);
/**
* Provides a lookup of ip addresses to node ids
* @param ips - nodes IPs

View File

@ -155,9 +155,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
}
try {
log.info("Terminating instance[%s]", instances);
amazonEC2Client.terminateInstances(
new TerminateInstancesRequest(
return terminateWithIds(
Lists.transform(
instances,
new Function<Instance, String>()
@ -169,22 +167,29 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
}
}
)
)
);
}
catch (Exception e) {
log.error(e, "Unable to terminate any instances.");
}
return null;
}
@Override
public AutoScalingData terminateWithIds(List<String> ids)
{
if (ids.isEmpty()) {
return new AutoScalingData(Lists.<String>newArrayList());
}
try {
log.info("Terminating instances[%s]", ids);
amazonEC2Client.terminateInstances(
new TerminateInstancesRequest(ids)
);
return new AutoScalingData(
Lists.transform(
ips,
new Function<String, String>()
{
@Override
public String apply(@Nullable String input)
{
return String.format("%s:%s", input, config.getWorkerPort());
}
}
)
);
return new AutoScalingData(ids);
}
catch (Exception e) {
log.error(e, "Unable to terminate any instances.");

View File

@ -44,6 +44,13 @@ public class NoopAutoScalingStrategy implements AutoScalingStrategy
return null;
}
@Override
public AutoScalingData terminateWithIds(List<String> ids)
{
log.info("If I were a real strategy I'd terminate %s now", ids);
return null;
}
@Override
public List<String> ipToIdLookup(List<String> ips)
{

View File

@ -132,8 +132,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
.addData("provisioningCount", currentlyProvisioning.size())
.emit();
List<String> nodeIps = autoScalingStrategy.idToIpLookup(Lists.newArrayList(currentlyProvisioning));
autoScalingStrategy.terminate(nodeIps);
autoScalingStrategy.terminateWithIds(Lists.newArrayList(currentlyProvisioning));
currentlyProvisioning.clear();
}
}

View File

@ -131,6 +131,6 @@ public class EC2AutoScalingStrategyTest
AutoScalingData deleted = strategy.terminate(Arrays.asList("dummyIP"));
Assert.assertEquals(deleted.getNodeIds().size(), 1);
Assert.assertEquals(String.format("%s:8080", IP), deleted.getNodeIds().get(0));
Assert.assertEquals(INSTANCE_ID, deleted.getNodeIds().get(0));
}
}

View File

@ -187,9 +187,7 @@ public class SimpleResourceManagementStrategyTest
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
.andReturn(Lists.<String>newArrayList()).times(2);
EasyMock.expect(autoScalingStrategy.idToIpLookup(EasyMock.<List<String>>anyObject()))
.andReturn(Lists.<String>newArrayList());
EasyMock.expect(autoScalingStrategy.terminate(EasyMock.<List<String>>anyObject()))
EasyMock.expect(autoScalingStrategy.terminateWithIds(EasyMock.<List<String>>anyObject()))
.andReturn(null);
EasyMock.expect(autoScalingStrategy.provision()).andReturn(
new AutoScalingData(Lists.<String>newArrayList("fake"))

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -23,7 +23,7 @@
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<packaging>pom</packaging>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
<name>druid</name>
<description>druid</description>
<scm>
@ -41,7 +41,7 @@
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.25.2</metamx.java-util.version>
<apache.curator.version>2.3.0</apache.curator.version>
<druid.api.version>0.1.7</druid.api.version>
<druid.api.version>0.1.8</druid.api.version>
</properties>
<modules>
@ -59,6 +59,7 @@
<module>kafka-seven</module>
<module>kafka-eight</module>
<module>rabbitmq</module>
<module>hll</module>
</modules>
<dependencyManagement>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -26,6 +26,7 @@ import com.metamx.common.guava.Sequence;
import io.druid.query.groupby.GroupByQuery;
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
import io.druid.query.search.search.SearchQuery;
import io.druid.query.select.SelectQuery;
import io.druid.query.spec.QuerySegmentSpec;
import io.druid.query.timeboundary.TimeBoundaryQuery;
import io.druid.query.timeseries.TimeseriesQuery;
@ -43,6 +44,7 @@ import java.util.Map;
@JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class),
@JsonSubTypes.Type(name = Query.GROUP_BY, value = GroupByQuery.class),
@JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class),
@JsonSubTypes.Type(name = Query.SELECT, value = SelectQuery.class),
@JsonSubTypes.Type(name = Query.TOPN, value = TopNQuery.class)
})
public interface Query<T>
@ -52,6 +54,7 @@ public interface Query<T>
public static final String TIME_BOUNDARY = "timeBoundary";
public static final String GROUP_BY = "groupBy";
public static final String SEGMENT_METADATA = "segmentMetadata";
public static final String SELECT = "select";
public static final String TOPN = "topN";
public DataSource getDataSource();

View File

@ -0,0 +1,117 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Maps;
import org.joda.time.DateTime;
import java.util.Map;
/**
*/
public class EventHolder
{
public static final String timestampKey = "timestamp";
private final String segmentId;
private final int offset;
private final Map<String, Object> event;
@JsonCreator
public EventHolder(
@JsonProperty("segmentId") String segmentId,
@JsonProperty("offset") int offset,
@JsonProperty("event") Map<String, Object> event
)
{
this.segmentId = segmentId;
this.offset = offset;
this.event = event;
}
public DateTime getTimestamp()
{
return (DateTime) event.get(timestampKey);
}
@JsonProperty
public String getSegmentId()
{
return segmentId;
}
@JsonProperty
public int getOffset()
{
return offset;
}
@JsonProperty
public Map<String, Object> getEvent()
{
return event;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
EventHolder that = (EventHolder) o;
if (offset != that.offset) {
return false;
}
if (!Maps.difference(event, ((EventHolder) o).event).areEqual()) {
return false;
}
if (segmentId != null ? !segmentId.equals(that.segmentId) : that.segmentId != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = segmentId != null ? segmentId.hashCode() : 0;
result = 31 * result + offset;
result = 31 * result + (event != null ? event.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "EventHolder{" +
"segmentId='" + segmentId + '\'' +
", offset=" + offset +
", event=" + event +
'}';
}
}

View File

@ -0,0 +1,100 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.primitives.Ints;
import java.nio.ByteBuffer;
import java.util.LinkedHashMap;
import java.util.Map;
/**
*/
public class PagingSpec
{
private final LinkedHashMap<String, Integer> pagingIdentifiers;
private final int threshold;
@JsonCreator
public PagingSpec(
@JsonProperty("pagingIdentifiers") LinkedHashMap<String, Integer> pagingIdentifiers,
@JsonProperty("threshold") int threshold
)
{
this.pagingIdentifiers = pagingIdentifiers;
this.threshold = threshold;
}
@JsonProperty
public Map<String, Integer> getPagingIdentifiers()
{
return pagingIdentifiers;
}
@JsonProperty
public int getThreshold()
{
return threshold;
}
public byte[] getCacheKey()
{
final byte[][] pagingKeys = new byte[pagingIdentifiers.size()][];
final byte[][] pagingValues = new byte[pagingIdentifiers.size()][];
int index = 0;
int pagingKeysSize = 0;
int pagingValuesSize = 0;
for (Map.Entry<String, Integer> entry : pagingIdentifiers.entrySet()) {
pagingKeys[index] = entry.getKey().getBytes();
pagingValues[index] = ByteBuffer.allocate(Ints.BYTES).putInt(entry.getValue()).array();
pagingKeysSize += pagingKeys[index].length;
pagingValuesSize += Ints.BYTES;
index++;
}
final byte[] thresholdBytes = ByteBuffer.allocate(Ints.BYTES).putInt(threshold).array();
final ByteBuffer queryCacheKey = ByteBuffer.allocate(pagingKeysSize + pagingValuesSize + thresholdBytes.length);
for (byte[] pagingKey : pagingKeys) {
queryCacheKey.put(pagingKey);
}
for (byte[] pagingValue : pagingValues) {
queryCacheKey.put(pagingValue);
}
queryCacheKey.put(thresholdBytes);
return queryCacheKey.array();
}
@Override
public String toString()
{
return "PagingSpec{" +
"pagingIdentifiers=" + pagingIdentifiers +
", threshold=" + threshold +
'}';
}
}

View File

@ -0,0 +1,77 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.metamx.common.guava.nary.BinaryFn;
import io.druid.granularity.AllGranularity;
import io.druid.granularity.QueryGranularity;
import io.druid.query.Result;
import org.joda.time.DateTime;
/**
*/
public class SelectBinaryFn
implements BinaryFn<Result<SelectResultValue>, Result<SelectResultValue>, Result<SelectResultValue>>
{
private final QueryGranularity gran;
private final PagingSpec pagingSpec;
public SelectBinaryFn(
QueryGranularity granularity,
PagingSpec pagingSpec
)
{
this.gran = granularity;
this.pagingSpec = pagingSpec;
}
@Override
public Result<SelectResultValue> apply(
Result<SelectResultValue> arg1, Result<SelectResultValue> arg2
)
{
if (arg1 == null) {
return arg2;
}
if (arg2 == null) {
return arg1;
}
final DateTime timestamp = (gran instanceof AllGranularity)
? arg1.getTimestamp()
: gran.toDateTime(gran.truncate(arg1.getTimestamp().getMillis()));
SelectResultValueBuilder builder = new SelectResultValueBuilder(timestamp, pagingSpec.getThreshold());
SelectResultValue arg1Val = arg1.getValue();
SelectResultValue arg2Val = arg2.getValue();
for (EventHolder event : arg1Val) {
builder.addEntry(event);
}
for (EventHolder event : arg2Val) {
builder.addEntry(event);
}
return builder.build();
}
}

View File

@ -0,0 +1,149 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import io.druid.granularity.QueryGranularity;
import io.druid.query.BaseQuery;
import io.druid.query.Query;
import io.druid.query.Result;
import io.druid.query.filter.DimFilter;
import io.druid.query.spec.QuerySegmentSpec;
import java.util.List;
import java.util.Map;
/**
*/
@JsonTypeName("select")
public class SelectQuery extends BaseQuery<Result<SelectResultValue>>
{
private final DimFilter dimFilter;
private final QueryGranularity granularity;
private final List<String> dimensions;
private final List<String> metrics;
private final PagingSpec pagingSpec;
@JsonCreator
public SelectQuery(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
@JsonProperty("filter") DimFilter dimFilter,
@JsonProperty("granularity") QueryGranularity granularity,
@JsonProperty("dimensions") List<String> dimensions,
@JsonProperty("metrics") List<String> metrics,
@JsonProperty("pagingSpec") PagingSpec pagingSpec,
@JsonProperty("context") Map<String, String> context
)
{
super(dataSource, querySegmentSpec, context);
this.dimFilter = dimFilter;
this.granularity = granularity;
this.dimensions = dimensions;
this.metrics = metrics;
this.pagingSpec = pagingSpec;
}
@Override
public boolean hasFilters()
{
return dimFilter != null;
}
@Override
public String getType()
{
return Query.SELECT;
}
@JsonProperty("filter")
public DimFilter getDimensionsFilter()
{
return dimFilter;
}
@JsonProperty
public QueryGranularity getGranularity()
{
return granularity;
}
@JsonProperty
public List<String> getDimensions()
{
return dimensions;
}
@JsonProperty
public PagingSpec getPagingSpec()
{
return pagingSpec;
}
@JsonProperty
public List<String> getMetrics()
{
return metrics;
}
public SelectQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
{
return new SelectQuery(
getDataSource(),
querySegmentSpec,
dimFilter,
granularity,
dimensions,
metrics,
pagingSpec,
getContext()
);
}
public SelectQuery withOverriddenContext(Map<String, String> contextOverrides)
{
return new SelectQuery(
getDataSource(),
getQuerySegmentSpec(),
dimFilter,
granularity,
dimensions,
metrics,
pagingSpec,
computeOverridenContext(contextOverrides)
);
}
@Override
public String toString()
{
return "SelectQuery{" +
"dataSource='" + getDataSource() + '\'' +
", querySegmentSpec=" + getQuerySegmentSpec() +
", dimFilter=" + dimFilter +
", granularity=" + granularity +
", dimensions=" + dimensions +
", metrics=" + metrics +
", pagingSpec=" + pagingSpec +
'}';
}
}

View File

@ -0,0 +1,167 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.metamx.common.guava.BaseSequence;
import com.metamx.common.guava.Sequence;
import io.druid.query.QueryRunnerHelper;
import io.druid.query.Result;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
import io.druid.segment.ObjectColumnSelector;
import io.druid.segment.Segment;
import io.druid.segment.StorageAdapter;
import io.druid.segment.TimestampColumnSelector;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.filter.Filters;
import org.joda.time.DateTime;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
*/
public class SelectQueryEngine
{
public Sequence<Result<SelectResultValue>> process(final SelectQuery query, final Segment segment)
{
return new BaseSequence<>(
new BaseSequence.IteratorMaker<Result<SelectResultValue>, Iterator<Result<SelectResultValue>>>()
{
@Override
public Iterator<Result<SelectResultValue>> make()
{
final StorageAdapter adapter = segment.asStorageAdapter();
final Iterable<String> dims;
if (query.getDimensions() == null || query.getDimensions().isEmpty()) {
dims = adapter.getAvailableDimensions();
} else {
dims = query.getDimensions();
}
final Iterable<String> metrics;
if (query.getMetrics() == null || query.getMetrics().isEmpty()) {
metrics = adapter.getAvailableMetrics();
} else {
metrics = query.getMetrics();
}
return QueryRunnerHelper.makeCursorBasedQuery(
adapter,
query.getQuerySegmentSpec().getIntervals(),
Filters.convertDimensionFilters(query.getDimensionsFilter()),
query.getGranularity(),
new Function<Cursor, Result<SelectResultValue>>()
{
@Override
public Result<SelectResultValue> apply(Cursor cursor)
{
final SelectResultValueBuilder builder = new SelectResultValueBuilder(
cursor.getTime(),
query.getPagingSpec()
.getThreshold()
);
final TimestampColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector();
final Map<String, DimensionSelector> dimSelectors = Maps.newHashMap();
for (String dim : dims) {
final DimensionSelector dimSelector = cursor.makeDimensionSelector(dim);
dimSelectors.put(dim, dimSelector);
}
final Map<String, ObjectColumnSelector> metSelectors = Maps.newHashMap();
for (String metric : metrics) {
final ObjectColumnSelector metricSelector = cursor.makeObjectColumnSelector(metric);
metSelectors.put(metric, metricSelector);
}
int startOffset;
if (query.getPagingSpec().getPagingIdentifiers() == null) {
startOffset = 0;
} else {
Integer offset = query.getPagingSpec().getPagingIdentifiers().get(segment.getIdentifier());
startOffset = (offset == null) ? 0 : offset;
}
cursor.advanceTo(startOffset);
int offset = 0;
while (!cursor.isDone() && offset < query.getPagingSpec().getThreshold()) {
final Map<String, Object> theEvent = Maps.newLinkedHashMap();
theEvent.put(EventHolder.timestampKey, new DateTime(timestampColumnSelector.getTimestamp()));
for (Map.Entry<String, DimensionSelector> dimSelector : dimSelectors.entrySet()) {
final String dim = dimSelector.getKey();
final DimensionSelector selector = dimSelector.getValue();
final IndexedInts vals = selector.getRow();
if (vals.size() == 1) {
final String dimVal = selector.lookupName(vals.get(0));
theEvent.put(dim, dimVal);
} else {
List<String> dimVals = Lists.newArrayList();
for (int i = 0; i < vals.size(); ++i) {
dimVals.add(selector.lookupName(vals.get(i)));
}
theEvent.put(dim, dimVals);
}
}
for (Map.Entry<String, ObjectColumnSelector> metSelector : metSelectors.entrySet()) {
final String metric = metSelector.getKey();
final ObjectColumnSelector selector = metSelector.getValue();
theEvent.put(metric, selector.get());
}
builder.addEntry(
new EventHolder(
segment.getIdentifier(),
startOffset + offset,
theEvent
)
);
cursor.advance();
offset++;
}
return builder.build();
}
}
).iterator();
}
@Override
public void cleanup(Iterator<Result<SelectResultValue>> toClean)
{
// https://github.com/metamx/druid/issues/128
while (toClean.hasNext()) {
toClean.next();
}
}
}
);
}
}

View File

@ -0,0 +1,291 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Functions;
import com.google.common.base.Joiner;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import com.metamx.common.guava.MergeSequence;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.nary.BinaryFn;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.collections.OrderedMergeSequence;
import io.druid.granularity.QueryGranularity;
import io.druid.query.CacheStrategy;
import io.druid.query.IntervalChunkingQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryConfig;
import io.druid.query.QueryRunner;
import io.druid.query.QueryToolChest;
import io.druid.query.Result;
import io.druid.query.ResultGranularTimestampComparator;
import io.druid.query.ResultMergeQueryRunner;
import io.druid.query.aggregation.MetricManipulationFn;
import io.druid.query.filter.DimFilter;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.Minutes;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
*/
public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResultValue>, SelectQuery>
{
private static final byte SELECT_QUERY = 0x13;
private static final Joiner COMMA_JOIN = Joiner.on(",");
private static final TypeReference<Object> OBJECT_TYPE_REFERENCE =
new TypeReference<Object>()
{
};
private static final TypeReference<Result<SelectResultValue>> TYPE_REFERENCE =
new TypeReference<Result<SelectResultValue>>()
{
};
private final QueryConfig config;
private final ObjectMapper jsonMapper;
@Inject
public SelectQueryQueryToolChest(QueryConfig config, ObjectMapper jsonMapper)
{
this.config = config;
this.jsonMapper = jsonMapper;
}
@Override
public QueryRunner<Result<SelectResultValue>> mergeResults(QueryRunner<Result<SelectResultValue>> queryRunner)
{
return new ResultMergeQueryRunner<Result<SelectResultValue>>(queryRunner)
{
@Override
protected Ordering<Result<SelectResultValue>> makeOrdering(Query<Result<SelectResultValue>> query)
{
return Ordering.from(
new ResultGranularTimestampComparator<SelectResultValue>(
((SelectQuery) query).getGranularity()
)
);
}
@Override
protected BinaryFn<Result<SelectResultValue>, Result<SelectResultValue>, Result<SelectResultValue>> createMergeFn(
Query<Result<SelectResultValue>> input
)
{
SelectQuery query = (SelectQuery) input;
return new SelectBinaryFn(
query.getGranularity(),
query.getPagingSpec()
);
}
};
}
@Override
public Sequence<Result<SelectResultValue>> mergeSequences(Sequence<Sequence<Result<SelectResultValue>>> seqOfSequences)
{
return new OrderedMergeSequence<Result<SelectResultValue>>(getOrdering(), seqOfSequences);
}
@Override
public ServiceMetricEvent.Builder makeMetricBuilder(SelectQuery query)
{
int numMinutes = 0;
for (Interval interval : query.getIntervals()) {
numMinutes += Minutes.minutesIn(interval).getMinutes();
}
return new ServiceMetricEvent.Builder()
.setUser2(query.getDataSource())
.setUser4("Select")
.setUser5(COMMA_JOIN.join(query.getIntervals()))
.setUser6(String.valueOf(query.hasFilters()))
.setUser9(Minutes.minutes(numMinutes).toString());
}
@Override
public Function<Result<SelectResultValue>, Result<SelectResultValue>> makeMetricManipulatorFn(
final SelectQuery query, final MetricManipulationFn fn
)
{
return Functions.identity();
}
@Override
public TypeReference<Result<SelectResultValue>> getResultTypeReference()
{
return TYPE_REFERENCE;
}
@Override
public CacheStrategy<Result<SelectResultValue>, Object, SelectQuery> getCacheStrategy(final SelectQuery query)
{
return new CacheStrategy<Result<SelectResultValue>, Object, SelectQuery>()
{
@Override
public byte[] computeCacheKey(SelectQuery query)
{
final DimFilter dimFilter = query.getDimensionsFilter();
final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey();
final byte[] granularityBytes = query.getGranularity().cacheKey();
final Set<String> dimensions = Sets.newTreeSet();
if (query.getDimensions() != null) {
dimensions.addAll(query.getDimensions());
}
final byte[][] dimensionsBytes = new byte[dimensions.size()][];
int dimensionsBytesSize = 0;
int index = 0;
for (String dimension : dimensions) {
dimensionsBytes[index] = dimension.getBytes();
dimensionsBytesSize += dimensionsBytes[index].length;
++index;
}
final Set<String> metrics = Sets.newTreeSet();
if (query.getMetrics() != null) {
dimensions.addAll(query.getMetrics());
}
final byte[][] metricBytes = new byte[metrics.size()][];
int metricBytesSize = 0;
index = 0;
for (String metric : metrics) {
metricBytes[index] = metric.getBytes();
metricBytesSize += metricBytes[index].length;
++index;
}
final ByteBuffer queryCacheKey = ByteBuffer
.allocate(
1
+ granularityBytes.length
+ filterBytes.length
+ query.getPagingSpec().getCacheKey().length
+ dimensionsBytesSize
+ metricBytesSize
)
.put(SELECT_QUERY)
.put(granularityBytes)
.put(filterBytes)
.put(query.getPagingSpec().getCacheKey());
for (byte[] dimensionsByte : dimensionsBytes) {
queryCacheKey.put(dimensionsByte);
}
for (byte[] metricByte : metricBytes) {
queryCacheKey.put(metricByte);
}
return queryCacheKey.array();
}
@Override
public TypeReference<Object> getCacheObjectClazz()
{
return OBJECT_TYPE_REFERENCE;
}
@Override
public Function<Result<SelectResultValue>, Object> prepareForCache()
{
return new Function<Result<SelectResultValue>, Object>()
{
@Override
public Object apply(final Result<SelectResultValue> input)
{
return Arrays.asList(
input.getTimestamp().getMillis(),
input.getValue().getPagingIdentifiers(),
input.getValue().getEvents()
);
}
};
}
@Override
public Function<Object, Result<SelectResultValue>> pullFromCache()
{
return new Function<Object, Result<SelectResultValue>>()
{
private final QueryGranularity granularity = query.getGranularity();
@Override
public Result<SelectResultValue> apply(Object input)
{
List<Object> results = (List<Object>) input;
Iterator<Object> resultIter = results.iterator();
DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue());
return new Result<SelectResultValue>(
timestamp,
new SelectResultValue(
(Map<String, Integer>) jsonMapper.convertValue(
resultIter.next(), new TypeReference<Map<String, Integer>>()
{
}
),
(List<EventHolder>) jsonMapper.convertValue(
resultIter.next(), new TypeReference<List<EventHolder>>()
{
}
)
)
);
}
};
}
@Override
public Sequence<Result<SelectResultValue>> mergeSequences(Sequence<Sequence<Result<SelectResultValue>>> seqOfSequences)
{
return new MergeSequence<Result<SelectResultValue>>(getOrdering(), seqOfSequences);
}
};
}
@Override
public QueryRunner<Result<SelectResultValue>> preMergeQueryDecoration(QueryRunner<Result<SelectResultValue>> runner)
{
return new IntervalChunkingQueryRunner<Result<SelectResultValue>>(runner, config.getChunkPeriod());
}
public Ordering<Result<SelectResultValue>> getOrdering()
{
return Ordering.natural();
}
}

View File

@ -0,0 +1,106 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.guava.Sequence;
import io.druid.query.ChainedExecutionQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryConfig;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
import io.druid.query.Result;
import io.druid.segment.Segment;
import java.util.concurrent.ExecutorService;
/**
*/
public class SelectQueryRunnerFactory
implements QueryRunnerFactory<Result<SelectResultValue>, SelectQuery>
{
public static SelectQueryRunnerFactory create(ObjectMapper jsonMapper)
{
return new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(new QueryConfig(), jsonMapper),
new SelectQueryEngine()
);
}
private final SelectQueryQueryToolChest toolChest;
private final SelectQueryEngine engine;
@Inject
public SelectQueryRunnerFactory(
SelectQueryQueryToolChest toolChest,
SelectQueryEngine engine
)
{
this.toolChest = toolChest;
this.engine = engine;
}
@Override
public QueryRunner<Result<SelectResultValue>> createRunner(final Segment segment)
{
return new SelectQueryRunner(engine, segment);
}
@Override
public QueryRunner<Result<SelectResultValue>> mergeRunners(
ExecutorService queryExecutor, Iterable<QueryRunner<Result<SelectResultValue>>> queryRunners
)
{
return new ChainedExecutionQueryRunner<Result<SelectResultValue>>(
queryExecutor, toolChest.getOrdering(), queryRunners
);
}
@Override
public QueryToolChest<Result<SelectResultValue>, SelectQuery> getToolchest()
{
return toolChest;
}
private static class SelectQueryRunner implements QueryRunner<Result<SelectResultValue>>
{
private final SelectQueryEngine engine;
private final Segment segment;
private SelectQueryRunner(SelectQueryEngine engine, Segment segment)
{
this.engine = engine;
this.segment = segment;
}
@Override
public Sequence<Result<SelectResultValue>> run(Query<Result<SelectResultValue>> input)
{
if (!(input instanceof SelectQuery)) {
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SelectQuery.class);
}
return engine.process((SelectQuery) input, segment);
}
}
}

View File

@ -0,0 +1,108 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
import com.metamx.common.ISE;
import javax.annotation.Nullable;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
*/
public class SelectResultValue implements Iterable<EventHolder>
{
private final Map<String, Integer> pagingIdentifiers;
private final List<EventHolder> events;
@JsonCreator
public SelectResultValue(
@JsonProperty("pagingIdentifiers") Map<String, Integer> pagingIdentifiers,
@JsonProperty("events") List<EventHolder> events)
{
this.pagingIdentifiers = pagingIdentifiers;
this.events = events;
}
@JsonProperty
public Map<String, Integer> getPagingIdentifiers()
{
return pagingIdentifiers;
}
@JsonProperty
public List<EventHolder> getEvents()
{
return events;
}
@Override
public Iterator<EventHolder> iterator()
{
return events.iterator();
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SelectResultValue that = (SelectResultValue) o;
if (events != null ? !events.equals(that.events) : that.events != null) {
return false;
}
if (pagingIdentifiers != null
? !pagingIdentifiers.equals(that.pagingIdentifiers)
: that.pagingIdentifiers != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = pagingIdentifiers != null ? pagingIdentifiers.hashCode() : 0;
result = 31 * result + (events != null ? events.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "SelectResultValue{" +
"pagingIdentifiers=" + pagingIdentifiers +
", events=" + events +
'}';
}
}

View File

@ -0,0 +1,98 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.MinMaxPriorityQueue;
import com.google.common.primitives.Longs;
import io.druid.query.Result;
import org.joda.time.DateTime;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
/**
*/
public class SelectResultValueBuilder
{
private static final Comparator<EventHolder> comparator = new Comparator<EventHolder>()
{
@Override
public int compare(EventHolder o1, EventHolder o2)
{
int retVal = Longs.compare(o1.getTimestamp().getMillis(), o2.getTimestamp().getMillis());
if (retVal == 0) {
retVal = o1.getSegmentId().compareTo(o2.getSegmentId());
}
if (retVal == 0) {
retVal = Integer.compare(o1.getOffset(), o2.getOffset());
}
return retVal;
}
};
private final DateTime timestamp;
private MinMaxPriorityQueue<EventHolder> pQueue = null;
public SelectResultValueBuilder(
DateTime timestamp,
int threshold
)
{
this.timestamp = timestamp;
instantiatePQueue(threshold, comparator);
}
public void addEntry(
EventHolder event
)
{
pQueue.add(event);
}
public Result<SelectResultValue> build()
{
// Pull out top aggregated values
List<EventHolder> values = Lists.newArrayListWithCapacity(pQueue.size());
Map<String, Integer> pagingIdentifiers = Maps.newLinkedHashMap();
while (!pQueue.isEmpty()) {
EventHolder event = pQueue.remove();
pagingIdentifiers.put(event.getSegmentId(), event.getOffset());
values.add(event);
}
return new Result<SelectResultValue>(
timestamp,
new SelectResultValue(pagingIdentifiers, values)
);
}
private void instantiatePQueue(int threshold, final Comparator comparator)
{
this.pQueue = MinMaxPriorityQueue.orderedBy(comparator).maximumSize(threshold).create();
}
}

View File

@ -24,6 +24,7 @@ package io.druid.segment;
*/
public interface ColumnSelectorFactory
{
public TimestampColumnSelector makeTimestampColumnSelector();
public DimensionSelector makeDimensionSelector(String dimensionName);
public FloatColumnSelector makeFloatColumnSelector(String columnName);
public ObjectColumnSelector makeObjectColumnSelector(String columnName);

View File

@ -21,10 +21,12 @@ package io.druid.segment;import org.joda.time.DateTime;
/**
*/
public interface Cursor extends ColumnSelectorFactory
{
public DateTime getTime();
public void advance();
public void advanceTo(int offset);
public boolean isDone();
public void reset();
}

View File

@ -30,18 +30,21 @@ import java.io.IOException;
public class IncrementalIndexSegment implements Segment
{
private final IncrementalIndex index;
private final String segmentIdentifier;
public IncrementalIndexSegment(
IncrementalIndex index
IncrementalIndex index,
String segmentIdentifier
)
{
this.index = index;
this.segmentIdentifier = segmentIdentifier;
}
@Override
public String getIdentifier()
{
throw new UnsupportedOperationException();
return segmentIdentifier;
}
@Override

View File

@ -23,6 +23,7 @@ import com.google.common.base.Function;
import com.google.common.base.Functions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.io.Closeables;
import com.metamx.common.collect.MoreIterators;
import com.metamx.common.guava.FunctionalIterable;
@ -77,6 +78,12 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
return index.getAvailableDimensions();
}
@Override
public Iterable<String> getAvailableMetrics()
{
return Sets.difference(Sets.newHashSet(index.getColumnNames()), Sets.newHashSet(index.getAvailableDimensions()));
}
@Override
public int getDimensionCardinality(String dimension)
{
@ -224,6 +231,16 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
cursorOffset.increment();
}
@Override
public void advanceTo(int offset)
{
int count = 0;
while (count < offset && !isDone()) {
advance();
count++;
}
}
@Override
public boolean isDone()
{
@ -236,6 +253,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
cursorOffset = initOffset.clone();
}
@Override
public TimestampColumnSelector makeTimestampColumnSelector()
{
return new TimestampColumnSelector()
{
@Override
public long getTimestamp()
{
return timestamps.getLongSingleValueRow(cursorOffset.getOffset());
}
};
}
@Override
public DimensionSelector makeDimensionSelector(String dimension)
{
@ -249,8 +279,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
if (column == null) {
return null;
}
else if (columnDesc.getCapabilities().hasMultipleValues()) {
} else if (columnDesc.getCapabilities().hasMultipleValues()) {
return new DimensionSelector()
{
@Override
@ -608,6 +637,12 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
++currRow;
}
@Override
public void advanceTo(int offset)
{
currRow += offset;
}
@Override
public boolean isDone()
{
@ -620,6 +655,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
currRow = initRow;
}
@Override
public TimestampColumnSelector makeTimestampColumnSelector()
{
return new TimestampColumnSelector()
{
@Override
public long getTimestamp()
{
return timestamps.getLongSingleValueRow(currRow);
}
};
}
@Override
public DimensionSelector makeDimensionSelector(String dimension)
{
@ -633,8 +681,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
if (dict == null) {
return null;
}
else if (column.getCapabilities().hasMultipleValues()) {
} else if (column.getCapabilities().hasMultipleValues()) {
return new DimensionSelector()
{
@Override

View File

@ -17,7 +17,9 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.segment;import io.druid.segment.data.Indexed;
package io.druid.segment;
import io.druid.segment.data.Indexed;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -28,6 +30,7 @@ public interface StorageAdapter extends CursorFactory
public String getSegmentIdentifier();
public Interval getInterval();
public Indexed<String> getAvailableDimensions();
public Iterable<String> getAvailableMetrics();
public int getDimensionCardinality(String dimension);
public DateTime getMinTime();
public DateTime getMaxTime();

View File

@ -0,0 +1,27 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.segment;
/**
*/
public interface TimestampColumnSelector
{
public long getTimestamp();
}

View File

@ -45,6 +45,7 @@ import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.DimensionSelector;
import io.druid.segment.FloatColumnSelector;
import io.druid.segment.ObjectColumnSelector;
import io.druid.segment.TimestampColumnSelector;
import io.druid.segment.serde.ComplexMetricExtractor;
import io.druid.segment.serde.ComplexMetricSerde;
import io.druid.segment.serde.ComplexMetrics;
@ -197,6 +198,19 @@ public class IncrementalIndex implements Iterable<Row>
aggs[i] = agg.factorize(
new ColumnSelectorFactory()
{
@Override
public TimestampColumnSelector makeTimestampColumnSelector()
{
return new TimestampColumnSelector()
{
@Override
public long getTimestamp()
{
return in.getTimestampFromEpoch();
}
};
}
@Override
public FloatColumnSelector makeFloatColumnSelector(String columnName)
{

View File

@ -1,3 +1,4 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
@ -38,6 +39,7 @@ import io.druid.segment.DimensionSelector;
import io.druid.segment.FloatColumnSelector;
import io.druid.segment.ObjectColumnSelector;
import io.druid.segment.StorageAdapter;
import io.druid.segment.TimestampColumnSelector;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.ListIndexed;
@ -87,6 +89,12 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
return new ListIndexed<String>(index.getDimensions(), String.class);
}
@Override
public Iterable<String> getAvailableMetrics()
{
return index.getMetricNames();
}
@Override
public int getDimensionCardinality(String dimension)
{
@ -205,6 +213,16 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
}
}
@Override
public void advanceTo(int offset)
{
int count = 0;
while (count < offset && !isDone()) {
advance();
count++;
}
}
@Override
public boolean isDone()
{
@ -237,6 +255,19 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
}
@Override
public TimestampColumnSelector makeTimestampColumnSelector()
{
return new TimestampColumnSelector()
{
@Override
public long getTimestamp()
{
return currEntry.getKey().getTimestamp();
}
};
}
@Override
public DimensionSelector makeDimensionSelector(String dimension)
{

View File

@ -48,11 +48,13 @@ import java.util.List;
*/
public class QueryRunnerTestHelper
{
public static final String segmentId= "testSegment";
public static final String dataSource = "testing";
public static final QueryGranularity dayGran = QueryGranularity.DAY;
public static final QueryGranularity allGran = QueryGranularity.ALL;
public static final String providerDimension = "proVider";
public static final String qualityDimension = "quality";
public static final String placementDimension = "placement";
public static final String placementishDimension = "placementish";
public static final String indexMetric = "index";
public static final String addRowsIndexConstantMetric = "addRowsIndexConstant";
@ -114,13 +116,13 @@ public class QueryRunnerTestHelper
return Arrays.asList(
new Object[][]{
{
makeQueryRunner(factory, new IncrementalIndexSegment(rtIndex))
makeQueryRunner(factory, new IncrementalIndexSegment(rtIndex, segmentId))
},
{
makeQueryRunner(factory, new QueryableIndexSegment(null, mMappedTestIndex))
makeQueryRunner(factory, new QueryableIndexSegment(segmentId, mMappedTestIndex))
},
{
makeQueryRunner(factory, new QueryableIndexSegment(null, mergedRealtimeIndex))
makeQueryRunner(factory, new QueryableIndexSegment(segmentId, mergedRealtimeIndex))
}
}
);

View File

@ -47,7 +47,7 @@ public class SegmentAnalyzerTest
public void testIncrementalDoesNotWork() throws Exception
{
final List<SegmentAnalysis> results = getSegmentAnalysises(
new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex())
new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), null)
);
Assert.assertEquals(0, results.size());

View File

@ -0,0 +1,224 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.metamx.common.ISE;
import io.druid.granularity.QueryGranularity;
import io.druid.query.Result;
import junit.framework.Assert;
import org.joda.time.DateTime;
import org.junit.Test;
import java.util.Arrays;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
/**
*/
public class SelectBinaryFnTest
{
private static final String segmentId1 = "testSegment";
private static final String segmentId2 = "testSegment";
@Test
public void testApply() throws Exception
{
SelectBinaryFn binaryFn = new SelectBinaryFn(QueryGranularity.ALL, new PagingSpec(null, 5));
Result<SelectResultValue> res1 = new Result<>(
new DateTime("2013-01-01"),
new SelectResultValue(
ImmutableMap.<String, Integer>of(),
Arrays.asList(
new EventHolder(
segmentId1,
0,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T00"),
"dim",
"first"
)
),
new EventHolder(
segmentId1,
1,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T03"),
"dim",
"fourth"
)
),
new EventHolder(
segmentId1,
2,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T05"),
"dim",
"sixth"
)
)
)
)
);
Result<SelectResultValue> res2 = new Result<>(
new DateTime("2013-01-01"),
new SelectResultValue(
ImmutableMap.<String, Integer>of(),
Arrays.asList(
new EventHolder(
segmentId2,
0,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T00"),
"dim",
"second"
)
),
new EventHolder(
segmentId2,
1,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T02"),
"dim",
"third"
)
),
new EventHolder(
segmentId2,
2,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T04"),
"dim",
"fifth"
)
)
)
)
);
Result<SelectResultValue> merged = binaryFn.apply(res1, res2);
Assert.assertEquals(res1.getTimestamp(), merged.getTimestamp());
LinkedHashMap<String, Integer> expectedPageIds = Maps.newLinkedHashMap();
expectedPageIds.put(segmentId1, 0);
expectedPageIds.put(segmentId2, 0);
expectedPageIds.put(segmentId2, 1);
expectedPageIds.put(segmentId1, 1);
expectedPageIds.put(segmentId2, 2);
Iterator<String> exSegmentIter = expectedPageIds.keySet().iterator();
Iterator<String> acSegmentIter = merged.getValue().getPagingIdentifiers().keySet().iterator();
verifyIters(exSegmentIter, acSegmentIter);
Iterator<Integer> exOffsetIter = expectedPageIds.values().iterator();
Iterator<Integer> acOffsetIter = merged.getValue().getPagingIdentifiers().values().iterator();
verifyIters(exOffsetIter, acOffsetIter);
List<EventHolder> exEvents = Arrays.<EventHolder>asList(
new EventHolder(
segmentId1,
0,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T00"), "dim", "first"
)
),
new EventHolder(
segmentId2,
0,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T00"),
"dim",
"second"
)
),
new EventHolder(
segmentId2,
1,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T02"),
"dim",
"third"
)
),
new EventHolder(
segmentId1,
1,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T03"),
"dim",
"fourth"
)
),
new EventHolder(
segmentId2,
2,
ImmutableMap.<String, Object>of(
EventHolder.timestampKey,
new DateTime("2013-01-01T04"),
"dim",
"fifth"
)
)
);
List<EventHolder> acEvents = merged.getValue().getEvents();
verifyEvents(exEvents, acEvents);
}
private void verifyIters(Iterator iter1, Iterator iter2)
{
while (iter1.hasNext()) {
Assert.assertEquals(iter1.next(), iter2.next());
}
if (iter2.hasNext()) {
throw new ISE("This should be empty!");
}
}
private void verifyEvents(List<EventHolder> events1, List<EventHolder> events2)
{
Iterator<EventHolder> ex = events1.iterator();
Iterator<EventHolder> ac = events2.iterator();
verifyIters(ex, ac);
}
}

View File

@ -0,0 +1,403 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.select;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.metamx.common.ISE;
import com.metamx.common.guava.Sequences;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.spec.LegacySegmentSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
*/
@RunWith(Parameterized.class)
public class SelectQueryRunnerTest
{
@Parameterized.Parameters
public static Collection<?> constructorFeeder() throws IOException
{
return QueryRunnerTestHelper.makeQueryRunners(
SelectQueryRunnerFactory.create(new DefaultObjectMapper())
);
}
private static final String providerLowercase = "provider";
private final QueryRunner runner;
public SelectQueryRunnerTest(
QueryRunner runner
)
{
this.runner = runner;
}
@Test
public void testFullOnSelect()
{
SelectQuery query = new SelectQuery(
QueryRunnerTestHelper.dataSource,
QueryRunnerTestHelper.fullOnInterval,
null,
QueryRunnerTestHelper.allGran,
Lists.<String>newArrayList(),
Lists.<String>newArrayList(),
new PagingSpec(null, 3),
null
);
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query),
Lists.<Result<SelectResultValue>>newArrayList()
);
List<Result<SelectResultValue>> expectedResults = Arrays.asList(
new Result<SelectResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
0,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(providerLowercase, "spot")
.put(QueryRunnerTestHelper.qualityDimension, "automotive")
.put(QueryRunnerTestHelper.placementDimension, "preferred")
.put(QueryRunnerTestHelper.placementishDimension, Lists.newArrayList("a", "preferred"))
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(providerLowercase, "spot")
.put(QueryRunnerTestHelper.qualityDimension, "business")
.put(QueryRunnerTestHelper.placementDimension, "preferred")
.put(QueryRunnerTestHelper.placementishDimension, Lists.newArrayList("b", "preferred"))
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(providerLowercase, "spot")
.put(QueryRunnerTestHelper.qualityDimension, "entertainment")
.put(QueryRunnerTestHelper.placementDimension, "preferred")
.put(QueryRunnerTestHelper.placementishDimension, Lists.newArrayList("e", "preferred"))
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
)
)
)
)
);
verify(expectedResults, results);
}
@Test
public void testSelectWithDimsAndMets()
{
SelectQuery query = new SelectQuery(
QueryRunnerTestHelper.dataSource,
QueryRunnerTestHelper.fullOnInterval,
null,
QueryRunnerTestHelper.allGran,
Lists.<String>newArrayList(providerLowercase),
Lists.<String>newArrayList(QueryRunnerTestHelper.indexMetric),
new PagingSpec(null, 3),
null
);
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query),
Lists.<Result<SelectResultValue>>newArrayList()
);
List<Result<SelectResultValue>> expectedResults = Arrays.asList(
new Result<SelectResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
0,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(providerLowercase, "spot")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(providerLowercase, "spot")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(providerLowercase, "spot")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
)
)
)
)
);
verify(expectedResults, results);
}
@Test
public void testSelectPagination()
{
SelectQuery query = new SelectQuery(
QueryRunnerTestHelper.dataSource,
QueryRunnerTestHelper.fullOnInterval,
null,
QueryRunnerTestHelper.allGran,
Lists.<String>newArrayList(QueryRunnerTestHelper.qualityDimension),
Lists.<String>newArrayList(QueryRunnerTestHelper.indexMetric),
new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3),
null
);
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query),
Lists.<Result<SelectResultValue>>newArrayList()
);
List<Result<SelectResultValue>> expectedResults = Arrays.asList(
new Result<SelectResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, 5),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
3,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(QueryRunnerTestHelper.qualityDimension, "health")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
4,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(QueryRunnerTestHelper.qualityDimension, "mezzanine")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
5,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(QueryRunnerTestHelper.qualityDimension, "news")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
)
)
)
)
);
verify(expectedResults, results);
}
@Test
public void testFullOnSelectWithFilter()
{
SelectQuery query = new SelectQuery(
QueryRunnerTestHelper.dataSource,
new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")),
new SelectorDimFilter(QueryRunnerTestHelper.providerDimension, "spot"),
QueryRunnerTestHelper.dayGran,
Lists.<String>newArrayList(QueryRunnerTestHelper.qualityDimension),
Lists.<String>newArrayList(QueryRunnerTestHelper.indexMetric),
new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3),
null
);
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query),
Lists.<Result<SelectResultValue>>newArrayList()
);
List<Result<SelectResultValue>> expectedResults = Arrays.asList(
new Result<SelectResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, 5),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
3,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(QueryRunnerTestHelper.qualityDimension, "health")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
4,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(QueryRunnerTestHelper.qualityDimension, "mezzanine")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
5,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put(QueryRunnerTestHelper.qualityDimension, "news")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.build()
)
)
)
),
new Result<SelectResultValue>(
new DateTime("2011-01-13T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, 5),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
3,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z"))
.put(QueryRunnerTestHelper.qualityDimension, "health")
.put(QueryRunnerTestHelper.indexMetric, 114.947403F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
4,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z"))
.put(QueryRunnerTestHelper.qualityDimension, "mezzanine")
.put(QueryRunnerTestHelper.indexMetric, 104.465767F)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
5,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z"))
.put(QueryRunnerTestHelper.qualityDimension, "news")
.put(QueryRunnerTestHelper.indexMetric, 102.851683F)
.build()
)
)
)
)
);
verify(expectedResults, results);
}
private static void verify(
Iterable<Result<SelectResultValue>> expectedResults,
Iterable<Result<SelectResultValue>> actualResults
)
{
Iterator<Result<SelectResultValue>> expectedIter = expectedResults.iterator();
Iterator<Result<SelectResultValue>> actualIter = actualResults.iterator();
while (expectedIter.hasNext()) {
Result<SelectResultValue> expected = expectedIter.next();
Result<SelectResultValue> actual = actualIter.next();
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
for (Map.Entry<String, Integer> entry : expected.getValue().getPagingIdentifiers().entrySet()) {
Assert.assertEquals(entry.getValue(), actual.getValue().getPagingIdentifiers().get(entry.getKey()));
}
Iterator<EventHolder> expectedEvts = expected.getValue().getEvents().iterator();
Iterator<EventHolder> actualEvts = actual.getValue().getEvents().iterator();
while (expectedEvts.hasNext()) {
EventHolder exHolder = expectedEvts.next();
EventHolder acHolder = actualEvts.next();
Assert.assertEquals(exHolder.getTimestamp(), acHolder.getTimestamp());
Assert.assertEquals(exHolder.getOffset(), acHolder.getOffset());
for (Map.Entry<String, Object> ex : exHolder.getEvent().entrySet()) {
Object actVal = acHolder.getEvent().get(ex.getKey());
// work around for current II limitations
if (acHolder.getEvent().get(ex.getKey()) instanceof Double) {
actVal = ((Double) actVal).floatValue();
}
Assert.assertEquals(ex.getValue(), actVal);
}
}
if (actualEvts.hasNext()) {
throw new ISE("This event iterator should be exhausted!");
}
}
if (actualIter.hasNext()) {
throw new ISE("This iterator should be exhausted!");
}
}
}

View File

@ -90,7 +90,7 @@ public class TimeseriesQueryRunnerBonusTest
final QueryRunnerFactory factory = TimeseriesQueryRunnerFactory.create();
final QueryRunner<Result<TimeseriesResultValue>> runner = makeQueryRunner(
factory,
new IncrementalIndexSegment(index)
new IncrementalIndexSegment(index, null)
);
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()

View File

@ -48,7 +48,7 @@ public class TopNQueryRunnerTestHelper
return Arrays.asList(
new Object[][]{
{
makeQueryRunner(factory, new IncrementalIndexSegment(rtIndex))
makeQueryRunner(factory, new IncrementalIndexSegment(rtIndex, null))
},
{
makeQueryRunner(factory, new QueryableIndexSegment(null, mMappedTestIndex))

View File

@ -83,7 +83,7 @@ public class SpatialFilterBonusTest
return Arrays.asList(
new Object[][]{
{
new IncrementalIndexSegment(rtIndex)
new IncrementalIndexSegment(rtIndex, null)
},
{
new QueryableIndexSegment(null, mMappedTestIndex)

View File

@ -83,7 +83,7 @@ public class SpatialFilterTest
return Arrays.asList(
new Object[][]{
{
new IncrementalIndexSegment(rtIndex)
new IncrementalIndexSegment(rtIndex, null)
},
{
new QueryableIndexSegment(null, mMappedTestIndex)

File diff suppressed because it is too large Load Diff

View File

@ -9,7 +9,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.52-SNAPSHOT</version>
<version>0.6.53-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -314,7 +314,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
final MultipleSpecificSegmentSpec segmentSpec = new MultipleSpecificSegmentSpec(descriptors);
List<Interval> intervals = segmentSpec.getIntervals();
if ("realtime".equals(server.getType()) || !populateCache || isBySegment) {
if (server.isRealtime() || !populateCache || isBySegment) {
resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec));
} else {
resultSeqToAdd = toolChest.mergeSequences(

View File

@ -36,7 +36,10 @@ import java.util.concurrent.ConcurrentMap;
*/
public class DruidServer implements Comparable
{
public static final int DEFAULT_PRIORITY = 0;
public static final int DEFAULT_NUM_REPLICANTS = 2;
public static final String DEFAULT_TIER = "_default_tier";
private static final Logger log = new Logger(DruidServer.class);
private final Object lock = new Object();
@ -59,7 +62,8 @@ public class DruidServer implements Comparable
node.getHost(),
config.getMaxSize(),
type,
config.getTier()
config.getTier(),
DEFAULT_PRIORITY
);
}
@ -69,10 +73,11 @@ public class DruidServer implements Comparable
@JsonProperty("host") String host,
@JsonProperty("maxSize") long maxSize,
@JsonProperty("type") String type,
@JsonProperty("tier") String tier
@JsonProperty("tier") String tier,
@JsonProperty("priority") int priority
)
{
this.metadata = new DruidServerMetadata(name, host, maxSize, type, tier);
this.metadata = new DruidServerMetadata(name, host, maxSize, type, tier, priority);
this.dataSources = new ConcurrentHashMap<String, DruidDataSource>();
this.segments = new ConcurrentHashMap<String, DataSegment>();
@ -118,6 +123,12 @@ public class DruidServer implements Comparable
return metadata.getTier();
}
@JsonProperty
public int getPriority()
{
return metadata.getPriority();
}
@JsonProperty
public Map<String, DataSegment> getSegments()
{
@ -125,6 +136,11 @@ public class DruidServer implements Comparable
return Collections.unmodifiableMap(segments);
}
public boolean isRealtime()
{
return getType().equalsIgnoreCase("realtime");
}
public DataSegment getSegment(String segmentName)
{
return segments.get(segmentName);

View File

@ -32,7 +32,10 @@ public class DruidServerConfig
private long maxSize = 0;
@JsonProperty
private String tier = "_default_tier";
private String tier = DruidServer.DEFAULT_TIER;
@JsonProperty
private int priority = DruidServer.DEFAULT_PRIORITY;
public long getMaxSize()
{
@ -43,4 +46,9 @@ public class DruidServerConfig
{
return tier;
}
public int getPriority()
{
return priority;
}
}

View File

@ -20,10 +20,14 @@
package io.druid.client.selector;
import com.google.common.primitives.Ints;
import com.metamx.common.ISE;
import io.druid.timeline.DataSegment;
import java.util.Collections;
import java.util.Comparator;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
public class ConnectionCountServerSelectorStrategy implements ServerSelectorStrategy
{
@ -37,8 +41,25 @@ public class ConnectionCountServerSelectorStrategy implements ServerSelectorStra
};
@Override
public QueryableDruidServer pick(Set<QueryableDruidServer> servers)
public QueryableDruidServer pick(
TreeMap<Integer, Set<QueryableDruidServer>> prioritizedServers, DataSegment segment
)
{
final Map.Entry<Integer, Set<QueryableDruidServer>> highestPriorityServers = prioritizedServers.pollLastEntry();
if (highestPriorityServers == null) {
return null;
}
final Set<QueryableDruidServer> servers = highestPriorityServers.getValue();
final int size = servers.size();
switch (size) {
case 0:
throw new ISE("[%s] Something hella weird going on here. We should not be here", segment.getIdentifier());
case 1:
return highestPriorityServers.getValue().iterator().next();
default:
return Collections.min(servers, comparator);
}
}
}

View File

@ -20,17 +20,36 @@
package io.druid.client.selector;
import com.google.common.collect.Iterators;
import com.metamx.common.ISE;
import io.druid.timeline.DataSegment;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.TreeMap;
public class RandomServerSelectorStrategy implements ServerSelectorStrategy
{
private static final Random random = new Random();
@Override
public QueryableDruidServer pick(Set<QueryableDruidServer> servers)
public QueryableDruidServer pick(TreeMap<Integer, Set<QueryableDruidServer>> prioritizedServers, DataSegment segment)
{
return Iterators.get(servers.iterator(), random.nextInt(servers.size()));
final Map.Entry<Integer, Set<QueryableDruidServer>> highestPriorityServers = prioritizedServers.pollLastEntry();
if (highestPriorityServers == null) {
return null;
}
final Set<QueryableDruidServer> servers = highestPriorityServers.getValue();
final int size = servers.size();
switch (size) {
case 0:
throw new ISE("[%s] Something hella weird going on here. We should not be here", segment.getIdentifier());
case 1:
return highestPriorityServers.getValue().iterator().next();
default:
return Iterators.get(servers.iterator(), random.nextInt(size));
}
}
}

View File

@ -19,18 +19,21 @@
package io.druid.client.selector;
import com.google.api.client.util.Maps;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import com.metamx.emitter.EmittingLogger;
import io.druid.timeline.DataSegment;
import java.util.Collections;
import java.util.Comparator;
import java.util.Set;
import java.util.TreeMap;
/**
*/
public class ServerSelector implements DiscoverySelector<QueryableDruidServer>
{
private static final EmittingLogger log = new EmittingLogger(ServerSelector.class);
private final Set<QueryableDruidServer> servers = Sets.newHashSet();
private final DataSegment segment;
@ -76,12 +79,17 @@ public class ServerSelector implements DiscoverySelector<QueryableDruidServer>
public QueryableDruidServer pick()
{
synchronized (this) {
final int size = servers.size();
switch (size) {
case 0: return null;
case 1: return servers.iterator().next();
default: return strategy.pick(servers);
TreeMap<Integer, Set<QueryableDruidServer>> prioritizedServers = Maps.newTreeMap();
for (QueryableDruidServer server : servers) {
Set<QueryableDruidServer> theServers = prioritizedServers.get(server.getServer().getPriority());
if (theServers == null) {
theServers = Sets.newHashSet();
prioritizedServers.put(server.getServer().getPriority(), theServers);
}
theServers.add(server);
}
return strategy.pick(prioritizedServers, segment);
}
}
}

View File

@ -21,8 +21,10 @@ package io.druid.client.selector;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import io.druid.timeline.DataSegment;
import java.util.Set;
import java.util.TreeMap;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = RandomServerSelectorStrategy.class)
@JsonSubTypes(value = {
@ -31,5 +33,5 @@ import java.util.Set;
})
public interface ServerSelectorStrategy
{
public QueryableDruidServer pick(Set<QueryableDruidServer> servers);
public QueryableDruidServer pick(TreeMap<Integer, Set<QueryableDruidServer>> prioritizedServers, DataSegment segment);
}

View File

@ -63,6 +63,7 @@ public class Announcer
private final PathChildrenCacheFactory factory;
private final List<Pair<String, byte[]>> toAnnounce = Lists.newArrayList();
private final List<Pair<String, byte[]>> toUpdate = 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>();
@ -92,6 +93,11 @@ public class Announcer
announce(pair.lhs, pair.rhs);
}
toAnnounce.clear();
for (Pair<String, byte[]> pair : toUpdate) {
update(pair.lhs, pair.rhs);
}
toUpdate.clear();
}
}
@ -268,6 +274,13 @@ public class Announcer
public void update(final String path, final byte[] bytes)
{
synchronized (toAnnounce) {
if (!started) {
toUpdate.add(Pair.of(path, bytes));
return;
}
}
final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
final String parentPath = pathAndNode.getPath();

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
@ -31,14 +32,14 @@ import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import io.druid.client.DruidServer;
import io.druid.concurrent.Execs;
import io.druid.guice.ManageLifecycle;
import io.druid.guice.annotations.Json;
import io.druid.server.coordinator.rules.PeriodLoadRule;
import io.druid.server.coordinator.rules.ForeverLoadRule;
import io.druid.server.coordinator.rules.Rule;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Period;
import org.skife.jdbi.v2.FoldController;
import org.skife.jdbi.v2.Folder3;
import org.skife.jdbi.v2.Handle;
@ -86,10 +87,11 @@ public class DatabaseRuleManager
}
final List<Rule> defaultRules = Arrays.<Rule>asList(
new PeriodLoadRule(
new Period("P5000Y"),
2,
"_default_tier"
new ForeverLoadRule(
ImmutableMap.<String, Integer>of(
DruidServer.DEFAULT_TIER,
DruidServer.DEFAULT_NUM_REPLICANTS
)
)
);
final String version = new DateTime().toString();

View File

@ -462,7 +462,8 @@ public class DatabaseSegmentManager
}
}
private String getSegmentsTable() {
private String getSegmentsTable()
{
return dbTables.get().getSegmentsTable();
}
}

View File

@ -31,6 +31,8 @@ import io.druid.query.metadata.SegmentMetadataQueryRunnerFactory;
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
import io.druid.query.search.SearchQueryRunnerFactory;
import io.druid.query.search.search.SearchQuery;
import io.druid.query.select.SelectQuery;
import io.druid.query.select.SelectQueryRunnerFactory;
import io.druid.query.timeboundary.TimeBoundaryQuery;
import io.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory;
import io.druid.query.timeseries.TimeseriesQuery;
@ -51,6 +53,7 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule
.put(TimeBoundaryQuery.class, TimeBoundaryQueryRunnerFactory.class)
.put(SegmentMetadataQuery.class, SegmentMetadataQueryRunnerFactory.class)
.put(GroupByQuery.class, GroupByQueryRunnerFactory.class)
.put(SelectQuery.class, SelectQueryRunnerFactory.class)
.put(TopNQuery.class, TopNQueryRunnerFactory.class)
.build();

View File

@ -34,6 +34,8 @@ import io.druid.query.metadata.metadata.SegmentMetadataQuery;
import io.druid.query.search.SearchQueryQueryToolChest;
import io.druid.query.search.search.SearchQuery;
import io.druid.query.search.search.SearchQueryConfig;
import io.druid.query.select.SelectQuery;
import io.druid.query.select.SelectQueryQueryToolChest;
import io.druid.query.timeboundary.TimeBoundaryQuery;
import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest;
import io.druid.query.timeseries.TimeseriesQuery;
@ -55,6 +57,7 @@ public class QueryToolChestModule implements Module
.put(TimeBoundaryQuery.class, TimeBoundaryQueryQueryToolChest.class)
.put(SegmentMetadataQuery.class, SegmentMetadataQueryQueryToolChest.class)
.put(GroupByQuery.class, GroupByQueryQueryToolChest.class)
.put(SelectQuery.class, SelectQueryQueryToolChest.class)
.put(TopNQuery.class, TopNQueryQueryToolChest.class)
.build();

View File

@ -67,7 +67,8 @@ public class StorageNodeModule implements Module
node.getHost(),
config.getMaxSize(),
nodeType.getNodeType(),
config.getTier()
config.getTier(),
config.getPriority()
);
}
}

View File

@ -28,17 +28,23 @@ import org.joda.time.Period;
*/
public class FireDepartmentConfig
{
private static int MAX_PENDING_PERSIST_BATCHES_DEFAULT = 2;
private final int maxRowsInMemory;
private final Period intermediatePersistPeriod;
private final int maxPendingPersists;
@JsonCreator
public FireDepartmentConfig(
@JsonProperty("maxRowsInMemory") int maxRowsInMemory,
@JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod
@JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod,
@JsonProperty("maxPendingPersists") int maxPendingPersists
)
{
this.maxRowsInMemory = maxRowsInMemory;
this.intermediatePersistPeriod = intermediatePersistPeriod;
this.maxPendingPersists = maxPendingPersists > 0
? maxPendingPersists
: MAX_PENDING_PERSIST_BATCHES_DEFAULT;
Preconditions.checkArgument(maxRowsInMemory > 0, "maxRowsInMemory[%s] should be greater than 0", maxRowsInMemory);
Preconditions.checkNotNull(intermediatePersistPeriod, "intermediatePersistPeriod");
@ -55,4 +61,10 @@ public class FireDepartmentConfig
{
return intermediatePersistPeriod;
}
@JsonProperty
public int getMaxPendingPersists()
{
return maxPendingPersists;
}
}

View File

@ -33,11 +33,12 @@ public class FireHydrant
public FireHydrant(
IncrementalIndex index,
int count
int count,
String segmentIdentifier
)
{
this.index = index;
this.adapter = new IncrementalIndexSegment(index);
this.adapter = new IncrementalIndexSegment(index, segmentIdentifier);
this.count = count;
}

View File

@ -0,0 +1,197 @@
package io.druid.segment.realtime.plumber;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.common.guava.ThreadRenamingCallable;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.IndexGranularity;
import io.druid.segment.realtime.FireDepartmentMetrics;
import io.druid.segment.realtime.Schema;
import io.druid.server.coordination.DataSegmentAnnouncer;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Period;
import java.io.File;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
/**
*/
public class FlushingPlumber extends RealtimePlumber
{
private static final EmittingLogger log = new EmittingLogger(FlushingPlumber.class);
private final Duration flushDuration;
private volatile ScheduledExecutorService flushScheduledExec = null;
private volatile boolean stopped = false;
public FlushingPlumber(
Duration flushDuration,
Period windowPeriod,
File basePersistDirectory,
IndexGranularity segmentGranularity,
Schema schema,
FireDepartmentMetrics metrics,
RejectionPolicy rejectionPolicy,
ServiceEmitter emitter,
QueryRunnerFactoryConglomerate conglomerate,
DataSegmentAnnouncer segmentAnnouncer,
ExecutorService queryExecutorService,
VersioningPolicy versioningPolicy,
int maxPendingPersists
)
{
super(
windowPeriod,
basePersistDirectory,
segmentGranularity,
schema,
metrics,
rejectionPolicy,
emitter,
conglomerate,
segmentAnnouncer,
queryExecutorService,
versioningPolicy,
null,
null,
null,
maxPendingPersists
);
this.flushDuration = flushDuration;
}
@Override
public void startJob()
{
log.info("Starting job for %s", getSchema().getDataSource());
computeBaseDir(getSchema()).mkdirs();
initializeExecutors();
if (flushScheduledExec == null) {
flushScheduledExec = Executors.newScheduledThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("flushing_scheduled_%d")
.build()
);
}
bootstrapSinksFromDisk();
startFlushThread();
}
protected void flushAfterDuration(final long truncatedTime, final Sink sink)
{
log.info(
"Abandoning segment %s at %s",
sink.getSegment().getIdentifier(),
new DateTime().plusMillis((int) flushDuration.getMillis())
);
ScheduledExecutors.scheduleWithFixedDelay(
flushScheduledExec,
flushDuration,
new Callable<ScheduledExecutors.Signal>()
{
@Override
public ScheduledExecutors.Signal call() throws Exception
{
log.info("Abandoning segment %s", sink.getSegment().getIdentifier());
abandonSegment(truncatedTime, sink);
return ScheduledExecutors.Signal.STOP;
}
}
);
}
private void startFlushThread()
{
final long truncatedNow = getSegmentGranularity().truncate(new DateTime()).getMillis();
final long windowMillis = getWindowPeriod().toStandardDuration().getMillis();
log.info(
"Expect to run at [%s]",
new DateTime().plus(
new Duration(System.currentTimeMillis(), getSegmentGranularity().increment(truncatedNow) + windowMillis)
)
);
ScheduledExecutors
.scheduleAtFixedRate(
flushScheduledExec,
new Duration(System.currentTimeMillis(), getSegmentGranularity().increment(truncatedNow) + windowMillis),
new Duration(truncatedNow, getSegmentGranularity().increment(truncatedNow)),
new ThreadRenamingCallable<ScheduledExecutors.Signal>(
String.format(
"%s-flusher-%d",
getSchema().getDataSource(),
getSchema().getShardSpec().getPartitionNum()
)
)
{
@Override
public ScheduledExecutors.Signal doCall()
{
if (stopped) {
log.info("Stopping flusher thread");
return ScheduledExecutors.Signal.STOP;
}
long minTimestamp = getSegmentGranularity().truncate(
getRejectionPolicy().getCurrMaxTime().minus(windowMillis)
).getMillis();
List<Map.Entry<Long, Sink>> sinksToPush = Lists.newArrayList();
for (Map.Entry<Long, Sink> entry : getSinks().entrySet()) {
final Long intervalStart = entry.getKey();
if (intervalStart < minTimestamp) {
log.info("Adding entry[%s] to flush.", entry);
sinksToPush.add(entry);
}
}
for (final Map.Entry<Long, Sink> entry : sinksToPush) {
flushAfterDuration(entry.getKey(), entry.getValue());
}
if (stopped) {
log.info("Stopping flusher thread");
return ScheduledExecutors.Signal.STOP;
} else {
return ScheduledExecutors.Signal.REPEAT;
}
}
}
);
}
@Override
public void finishJob()
{
log.info("Stopping job");
for (final Map.Entry<Long, Sink> entry : getSinks().entrySet()) {
abandonSegment(entry.getKey(), entry.getValue());
}
shutdownExecutors();
if (flushScheduledExec != null) {
flushScheduledExec.shutdown();
}
stopped = true;
}
}

View File

@ -0,0 +1,131 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.segment.realtime.plumber;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.guice.annotations.Processing;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.IndexGranularity;
import io.druid.segment.realtime.FireDepartmentMetrics;
import io.druid.segment.realtime.Schema;
import io.druid.server.coordination.DataSegmentAnnouncer;
import org.joda.time.Duration;
import org.joda.time.Period;
import javax.validation.constraints.NotNull;
import java.io.File;
import java.util.concurrent.ExecutorService;
/**
* This plumber just drops segments at the end of a flush duration instead of handing them off. It is only useful if you want to run
* a real time node without the rest of the Druid cluster.
*/
public class FlushingPlumberSchool implements PlumberSchool
{
private static final EmittingLogger log = new EmittingLogger(FlushingPlumberSchool.class);
private final Duration flushDuration;
private final Period windowPeriod;
private final File basePersistDirectory;
private final IndexGranularity segmentGranularity;
private final int maxPendingPersists;
@JacksonInject
@NotNull
private volatile ServiceEmitter emitter;
@JacksonInject
@NotNull
private volatile QueryRunnerFactoryConglomerate conglomerate = null;
@JacksonInject
@NotNull
private volatile DataSegmentAnnouncer segmentAnnouncer = null;
@JacksonInject
@NotNull
@Processing
private volatile ExecutorService queryExecutorService = null;
private volatile VersioningPolicy versioningPolicy = null;
private volatile RejectionPolicyFactory rejectionPolicyFactory = null;
@JsonCreator
public FlushingPlumberSchool(
@JsonProperty("flushDuration") Duration flushDuration,
@JsonProperty("windowPeriod") Period windowPeriod,
@JsonProperty("basePersistDirectory") File basePersistDirectory,
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity,
@JsonProperty("maxPendingPersists") int maxPendingPersists
)
{
this.flushDuration = flushDuration;
this.windowPeriod = windowPeriod;
this.basePersistDirectory = basePersistDirectory;
this.segmentGranularity = segmentGranularity;
this.versioningPolicy = new IntervalStartVersioningPolicy();
this.rejectionPolicyFactory = new ServerTimeRejectionPolicyFactory();
this.maxPendingPersists = maxPendingPersists;
Preconditions.checkArgument(maxPendingPersists > 0, "FlushingPlumberSchool requires maxPendingPersists > 0");
Preconditions.checkNotNull(flushDuration, "FlushingPlumberSchool requires a flushDuration.");
Preconditions.checkNotNull(windowPeriod, "FlushingPlumberSchool requires a windowPeriod.");
Preconditions.checkNotNull(basePersistDirectory, "FlushingPlumberSchool requires a basePersistDirectory.");
Preconditions.checkNotNull(segmentGranularity, "FlushingPlumberSchool requires a segmentGranularity.");
}
@Override
public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics)
{
verifyState();
final RejectionPolicy rejectionPolicy = rejectionPolicyFactory.create(windowPeriod);
log.info("Creating plumber using rejectionPolicy[%s]", rejectionPolicy);
return new FlushingPlumber(
flushDuration,
windowPeriod,
basePersistDirectory,
segmentGranularity,
schema,
metrics,
rejectionPolicy,
emitter,
conglomerate,
segmentAnnouncer,
queryExecutorService,
versioningPolicy,
maxPendingPersists
);
}
private void verifyState()
{
Preconditions.checkNotNull(conglomerate, "must specify a queryRunnerFactoryConglomerate to do this action.");
Preconditions.checkNotNull(segmentAnnouncer, "must specify a segmentAnnouncer to do this action.");
Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action.");
}
}

View File

@ -27,9 +27,10 @@ import io.druid.segment.realtime.Schema;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(
@JsonSubTypes.Type(name = "realtime", value = RealtimePlumberSchool.class)
)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "realtime", value = RealtimePlumberSchool.class),
@JsonSubTypes.Type(name = "flushing", value = FlushingPlumberSchool.class)
})
public interface PlumberSchool
{
/**

View File

@ -0,0 +1,735 @@
package io.druid.segment.realtime.plumber;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.Pair;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.client.DruidServer;
import io.druid.client.ServerView;
import io.druid.common.guava.ThreadRenamingCallable;
import io.druid.common.guava.ThreadRenamingRunnable;
import io.druid.concurrent.Execs;
import io.druid.query.MetricsEmittingQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.query.QueryToolChest;
import io.druid.query.SegmentDescriptor;
import io.druid.query.spec.SpecificSegmentQueryRunner;
import io.druid.query.spec.SpecificSegmentSpec;
import io.druid.segment.IndexGranularity;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.Segment;
import io.druid.segment.loading.DataSegmentPusher;
import io.druid.segment.realtime.FireDepartmentMetrics;
import io.druid.segment.realtime.FireHydrant;
import io.druid.segment.realtime.Schema;
import io.druid.segment.realtime.SegmentPublisher;
import io.druid.server.coordination.DataSegmentAnnouncer;
import io.druid.timeline.DataSegment;
import io.druid.timeline.TimelineObjectHolder;
import io.druid.timeline.VersionedIntervalTimeline;
import io.druid.timeline.partition.SingleElementPartitionChunk;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Interval;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.io.File;
import java.io.FilenameFilter;
import java.io.IOException;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
/**
*/
public class RealtimePlumber implements Plumber
{
private static final EmittingLogger log = new EmittingLogger(RealtimePlumber.class);
private final Period windowPeriod;
private final File basePersistDirectory;
private final IndexGranularity segmentGranularity;
private final Schema schema;
private final FireDepartmentMetrics metrics;
private final RejectionPolicy rejectionPolicy;
private final ServiceEmitter emitter;
private final QueryRunnerFactoryConglomerate conglomerate;
private final DataSegmentAnnouncer segmentAnnouncer;
private final ExecutorService queryExecutorService;
private final VersioningPolicy versioningPolicy;
private final DataSegmentPusher dataSegmentPusher;
private final SegmentPublisher segmentPublisher;
private final ServerView serverView;
private final int maxPendingPersists;
private final Object handoffCondition = new Object();
private final Map<Long, Sink> sinks = Maps.newConcurrentMap();
private final VersionedIntervalTimeline<String, Sink> sinkTimeline = new VersionedIntervalTimeline<String, Sink>(
String.CASE_INSENSITIVE_ORDER
);
private volatile boolean shuttingDown = false;
private volatile boolean stopped = false;
private volatile ExecutorService persistExecutor = null;
private volatile ScheduledExecutorService scheduledExecutor = null;
public RealtimePlumber(
Period windowPeriod,
File basePersistDirectory,
IndexGranularity segmentGranularity,
Schema schema,
FireDepartmentMetrics metrics,
RejectionPolicy rejectionPolicy,
ServiceEmitter emitter,
QueryRunnerFactoryConglomerate conglomerate,
DataSegmentAnnouncer segmentAnnouncer,
ExecutorService queryExecutorService,
VersioningPolicy versioningPolicy,
DataSegmentPusher dataSegmentPusher,
SegmentPublisher segmentPublisher,
ServerView serverView,
int maxPendingPersists
)
{
this.windowPeriod = windowPeriod;
this.basePersistDirectory = basePersistDirectory;
this.segmentGranularity = segmentGranularity;
this.schema = schema;
this.metrics = metrics;
this.rejectionPolicy = rejectionPolicy;
this.emitter = emitter;
this.conglomerate = conglomerate;
this.segmentAnnouncer = segmentAnnouncer;
this.queryExecutorService = queryExecutorService;
this.versioningPolicy = versioningPolicy;
this.dataSegmentPusher = dataSegmentPusher;
this.segmentPublisher = segmentPublisher;
this.serverView = serverView;
this.maxPendingPersists = maxPendingPersists;
}
public Schema getSchema()
{
return schema;
}
public Period getWindowPeriod()
{
return windowPeriod;
}
public IndexGranularity getSegmentGranularity()
{
return segmentGranularity;
}
public VersioningPolicy getVersioningPolicy()
{
return versioningPolicy;
}
public RejectionPolicy getRejectionPolicy()
{
return rejectionPolicy;
}
public Map<Long, Sink> getSinks()
{
return sinks;
}
@Override
public void startJob()
{
computeBaseDir(schema).mkdirs();
initializeExecutors();
bootstrapSinksFromDisk();
registerServerViewCallback();
startPersistThread();
}
@Override
public Sink getSink(long timestamp)
{
if (!rejectionPolicy.accept(timestamp)) {
return null;
}
final long truncatedTime = segmentGranularity.truncate(timestamp);
Sink retVal = sinks.get(truncatedTime);
if (retVal == null) {
final Interval sinkInterval = new Interval(
new DateTime(truncatedTime),
segmentGranularity.increment(new DateTime(truncatedTime))
);
retVal = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval));
try {
segmentAnnouncer.announceSegment(retVal.getSegment());
sinks.put(truncatedTime, retVal);
sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), new SingleElementPartitionChunk<Sink>(retVal));
}
catch (IOException e) {
log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource())
.addData("interval", retVal.getInterval())
.emit();
}
}
return retVal;
}
@Override
public <T> QueryRunner<T> getQueryRunner(final Query<T> query)
{
final QueryRunnerFactory<T, Query<T>> factory = conglomerate.findFactory(query);
final QueryToolChest<T, Query<T>> toolchest = factory.getToolchest();
final Function<Query<T>, ServiceMetricEvent.Builder> builderFn =
new Function<Query<T>, ServiceMetricEvent.Builder>()
{
@Override
public ServiceMetricEvent.Builder apply(@Nullable Query<T> input)
{
return toolchest.makeMetricBuilder(query);
}
};
List<TimelineObjectHolder<String, Sink>> querySinks = Lists.newArrayList();
for (Interval interval : query.getIntervals()) {
querySinks.addAll(sinkTimeline.lookup(interval));
}
return toolchest.mergeResults(
factory.mergeRunners(
queryExecutorService,
FunctionalIterable
.create(querySinks)
.transform(
new Function<TimelineObjectHolder<String, Sink>, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(TimelineObjectHolder<String, Sink> holder)
{
final Sink theSink = holder.getObject().getChunk(0).getObject();
return new SpecificSegmentQueryRunner<T>(
new MetricsEmittingQueryRunner<T>(
emitter,
builderFn,
factory.mergeRunners(
MoreExecutors.sameThreadExecutor(),
Iterables.transform(
theSink,
new Function<FireHydrant, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(FireHydrant input)
{
return factory.createRunner(input.getSegment());
}
}
)
)
),
new SpecificSegmentSpec(
new SegmentDescriptor(
holder.getInterval(),
theSink.getSegment().getVersion(),
theSink.getSegment().getShardSpec().getPartitionNum()
)
)
);
}
}
)
)
);
}
@Override
public void persist(final Runnable commitRunnable)
{
final List<Pair<FireHydrant, Interval>> indexesToPersist = Lists.newArrayList();
for (Sink sink : sinks.values()) {
if (sink.swappable()) {
indexesToPersist.add(Pair.of(sink.swap(), sink.getInterval()));
}
}
log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource());
persistExecutor.execute(
new ThreadRenamingRunnable(String.format("%s-incremental-persist", schema.getDataSource()))
{
@Override
public void doRun()
{
for (Pair<FireHydrant, Interval> pair : indexesToPersist) {
metrics.incrementRowOutputCount(persistHydrant(pair.lhs, schema, pair.rhs));
}
commitRunnable.run();
}
}
);
}
// Submits persist-n-merge task for a Sink to the persistExecutor
private void persistAndMerge(final long truncatedTime, final Sink sink)
{
final String threadName = String.format(
"%s-%s-persist-n-merge", schema.getDataSource(), new DateTime(truncatedTime)
);
persistExecutor.execute(
new ThreadRenamingRunnable(threadName)
{
@Override
public void doRun()
{
final Interval interval = sink.getInterval();
for (FireHydrant hydrant : sink) {
if (!hydrant.hasSwapped()) {
log.info("Hydrant[%s] hasn't swapped yet, swapping. Sink[%s]", hydrant, sink);
final int rowCount = persistHydrant(hydrant, schema, interval);
metrics.incrementRowOutputCount(rowCount);
}
}
final File mergedTarget = new File(computePersistDir(schema, interval), "merged");
if (mergedTarget.exists()) {
log.info("Skipping already-merged sink: %s", sink);
return;
}
File mergedFile = null;
try {
List<QueryableIndex> indexes = Lists.newArrayList();
for (FireHydrant fireHydrant : sink) {
Segment segment = fireHydrant.getSegment();
final QueryableIndex queryableIndex = segment.asQueryableIndex();
log.info("Adding hydrant[%s]", fireHydrant);
indexes.add(queryableIndex);
}
mergedFile = IndexMerger.mergeQueryableIndex(
indexes,
schema.getAggregators(),
mergedTarget
);
QueryableIndex index = IndexIO.loadIndex(mergedFile);
DataSegment segment = dataSegmentPusher.push(
mergedFile,
sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions()))
);
segmentPublisher.publishSegment(segment);
}
catch (IOException e) {
log.makeAlert(e, "Failed to persist merged index[%s]", schema.getDataSource())
.addData("interval", interval)
.emit();
if (shuttingDown) {
// We're trying to shut down, and this segment failed to push. Let's just get rid of it.
abandonSegment(truncatedTime, sink);
}
}
if (mergedFile != null) {
try {
log.info("Deleting Index File[%s]", mergedFile);
FileUtils.deleteDirectory(mergedFile);
}
catch (IOException e) {
log.warn(e, "Error deleting directory[%s]", mergedFile);
}
}
}
}
);
}
@Override
public void finishJob()
{
log.info("Shutting down...");
shuttingDown = true;
for (final Map.Entry<Long, Sink> entry : sinks.entrySet()) {
persistAndMerge(entry.getKey(), entry.getValue());
}
while (!sinks.isEmpty()) {
try {
log.info(
"Cannot shut down yet! Sinks remaining: %s",
Joiner.on(", ").join(
Iterables.transform(
sinks.values(),
new Function<Sink, String>()
{
@Override
public String apply(Sink input)
{
return input.getSegment().getIdentifier();
}
}
)
)
);
synchronized (handoffCondition) {
while (!sinks.isEmpty()) {
handoffCondition.wait();
}
}
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
}
}
shutdownExecutors();
stopped = true;
}
protected void initializeExecutors()
{
if (persistExecutor == null) {
// use a blocking single threaded executor to throttle the firehose when write to disk is slow
persistExecutor = Execs.newBlockingSingleThreaded(
"plumber_persist_%d", maxPendingPersists
);
}
if (scheduledExecutor == null) {
scheduledExecutor = Executors.newScheduledThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("plumber_scheduled_%d")
.build()
);
}
}
protected void shutdownExecutors()
{
// scheduledExecutor is shutdown here, but persistExecutor is shutdown when the
// ServerView sends it a new segment callback
if (scheduledExecutor != null) {
scheduledExecutor.shutdown();
}
}
protected void bootstrapSinksFromDisk()
{
File baseDir = computeBaseDir(schema);
if (baseDir == null || !baseDir.exists()) {
return;
}
File[] files = baseDir.listFiles();
if (files == null) {
return;
}
for (File sinkDir : files) {
Interval sinkInterval = new Interval(sinkDir.getName().replace("_", "/"));
//final File[] sinkFiles = sinkDir.listFiles();
// To avoid reading and listing of "merged" dir
final File[] sinkFiles = sinkDir.listFiles(
new FilenameFilter()
{
@Override
public boolean accept(File dir, String fileName)
{
return !(Ints.tryParse(fileName) == null);
}
}
);
Arrays.sort(
sinkFiles,
new Comparator<File>()
{
@Override
public int compare(File o1, File o2)
{
try {
return Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName()));
}
catch (NumberFormatException e) {
log.error(e, "Couldn't compare as numbers? [%s][%s]", o1, o2);
return o1.compareTo(o2);
}
}
}
);
try {
List<FireHydrant> hydrants = Lists.newArrayList();
for (File segmentDir : sinkFiles) {
log.info("Loading previously persisted segment at [%s]", segmentDir);
// Although this has been tackled at start of this method.
// Just a doubly-check added to skip "merged" dir. from being added to hydrants
// If 100% sure that this is not needed, this check can be removed.
if (Ints.tryParse(segmentDir.getName()) == null) {
continue;
}
hydrants.add(
new FireHydrant(
new QueryableIndexSegment(
DataSegment.makeDataSegmentIdentifier(
schema.getDataSource(),
sinkInterval.getStart(),
sinkInterval.getEnd(),
versioningPolicy.getVersion(sinkInterval),
schema.getShardSpec()
),
IndexIO.loadIndex(segmentDir)
),
Integer.parseInt(segmentDir.getName())
)
);
}
Sink currSink = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval), hydrants);
sinks.put(sinkInterval.getStartMillis(), currSink);
sinkTimeline.add(
currSink.getInterval(),
currSink.getVersion(),
new SingleElementPartitionChunk<Sink>(currSink)
);
segmentAnnouncer.announceSegment(currSink.getSegment());
}
catch (IOException e) {
log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource())
.addData("interval", sinkInterval)
.emit();
}
}
}
protected void startPersistThread()
{
final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis();
final long windowMillis = windowPeriod.toStandardDuration().getMillis();
log.info(
"Expect to run at [%s]",
new DateTime().plus(
new Duration(System.currentTimeMillis(), segmentGranularity.increment(truncatedNow) + windowMillis)
)
);
ScheduledExecutors
.scheduleAtFixedRate(
scheduledExecutor,
new Duration(System.currentTimeMillis(), segmentGranularity.increment(truncatedNow) + windowMillis),
new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)),
new ThreadRenamingCallable<ScheduledExecutors.Signal>(
String.format(
"%s-overseer-%d",
schema.getDataSource(),
schema.getShardSpec().getPartitionNum()
)
)
{
@Override
public ScheduledExecutors.Signal doCall()
{
if (stopped) {
log.info("Stopping merge-n-push overseer thread");
return ScheduledExecutors.Signal.STOP;
}
log.info("Starting merge and push.");
long minTimestamp = segmentGranularity.truncate(
rejectionPolicy.getCurrMaxTime().minus(windowMillis)
).getMillis();
List<Map.Entry<Long, Sink>> sinksToPush = Lists.newArrayList();
for (Map.Entry<Long, Sink> entry : sinks.entrySet()) {
final Long intervalStart = entry.getKey();
if (intervalStart < minTimestamp) {
log.info("Adding entry[%s] for merge and push.", entry);
sinksToPush.add(entry);
}
}
for (final Map.Entry<Long, Sink> entry : sinksToPush) {
persistAndMerge(entry.getKey(), entry.getValue());
}
if (stopped) {
log.info("Stopping merge-n-push overseer thread");
return ScheduledExecutors.Signal.STOP;
} else {
return ScheduledExecutors.Signal.REPEAT;
}
}
}
);
}
/**
* Unannounces a given sink and removes all local references to it.
*/
protected void abandonSegment(final long truncatedTime, final Sink sink)
{
try {
segmentAnnouncer.unannounceSegment(sink.getSegment());
FileUtils.deleteDirectory(computePersistDir(schema, sink.getInterval()));
log.info("Removing sinkKey %d for segment %s", truncatedTime, sink.getSegment().getIdentifier());
sinks.remove(truncatedTime);
sinkTimeline.remove(
sink.getInterval(),
sink.getVersion(),
new SingleElementPartitionChunk<>(sink)
);
synchronized (handoffCondition) {
handoffCondition.notifyAll();
}
}
catch (IOException e) {
log.makeAlert(e, "Unable to abandon old segment for dataSource[%s]", schema.getDataSource())
.addData("interval", sink.getInterval())
.emit();
}
}
protected File computeBaseDir(Schema schema)
{
return new File(basePersistDirectory, schema.getDataSource());
}
protected File computePersistDir(Schema schema, Interval interval)
{
return new File(computeBaseDir(schema), interval.toString().replace("/", "_"));
}
/**
* Persists the given hydrant and returns the number of rows persisted
*
* @param indexToPersist
* @param schema
* @param interval
*
* @return the number of rows persisted
*/
protected int persistHydrant(FireHydrant indexToPersist, Schema schema, Interval interval)
{
if (indexToPersist.hasSwapped()) {
log.info(
"DataSource[%s], Interval[%s], Hydrant[%s] already swapped. Ignoring request to persist.",
schema.getDataSource(), interval, indexToPersist
);
return 0;
}
log.info("DataSource[%s], Interval[%s], persisting Hydrant[%s]", schema.getDataSource(), interval, indexToPersist);
try {
int numRows = indexToPersist.getIndex().size();
File persistedFile = IndexMerger.persist(
indexToPersist.getIndex(),
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount()))
);
indexToPersist.swapSegment(
new QueryableIndexSegment(
indexToPersist.getSegment().getIdentifier(),
IndexIO.loadIndex(persistedFile)
)
);
return numRows;
}
catch (IOException e) {
log.makeAlert("dataSource[%s] -- incremental persist failed", schema.getDataSource())
.addData("interval", interval)
.addData("count", indexToPersist.getCount())
.emit();
throw Throwables.propagate(e);
}
}
private void registerServerViewCallback()
{
serverView.registerSegmentCallback(
persistExecutor,
new ServerView.BaseSegmentCallback()
{
@Override
public ServerView.CallbackAction segmentAdded(DruidServer server, DataSegment segment)
{
if (stopped) {
log.info("Unregistering ServerViewCallback");
persistExecutor.shutdown();
return ServerView.CallbackAction.UNREGISTER;
}
if (server.isRealtime()) {
return ServerView.CallbackAction.CONTINUE;
}
log.debug("Checking segment[%s] on server[%s]", segment, server);
if (schema.getDataSource().equals(segment.getDataSource())) {
final Interval interval = segment.getInterval();
for (Map.Entry<Long, Sink> entry : sinks.entrySet()) {
final Long sinkKey = entry.getKey();
if (interval.contains(sinkKey)) {
final Sink sink = entry.getValue();
log.info("Segment[%s] matches sink[%s] on server[%s]", segment, sink, server);
final String segmentVersion = segment.getVersion();
final String sinkVersion = sink.getSegment().getVersion();
if (segmentVersion.compareTo(sinkVersion) >= 0) {
log.info("Segment version[%s] >= sink version[%s]", segmentVersion, sinkVersion);
abandonSegment(sinkKey, sink);
}
}
}
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
}
}

View File

@ -22,113 +22,58 @@ package io.druid.segment.realtime.plumber;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.Pair;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.client.DruidServer;
import io.druid.client.ServerView;
import io.druid.common.guava.ThreadRenamingCallable;
import io.druid.common.guava.ThreadRenamingRunnable;
import io.druid.guice.annotations.Processing;
import io.druid.query.MetricsEmittingQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.query.QueryToolChest;
import io.druid.query.SegmentDescriptor;
import io.druid.query.spec.SpecificSegmentQueryRunner;
import io.druid.query.spec.SpecificSegmentSpec;
import io.druid.segment.IndexGranularity;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.Segment;
import io.druid.segment.loading.DataSegmentPusher;
import io.druid.segment.realtime.FireDepartmentMetrics;
import io.druid.segment.realtime.FireHydrant;
import io.druid.segment.realtime.Schema;
import io.druid.segment.realtime.SegmentPublisher;
import io.druid.server.coordination.DataSegmentAnnouncer;
import io.druid.timeline.DataSegment;
import io.druid.timeline.TimelineObjectHolder;
import io.druid.timeline.VersionedIntervalTimeline;
import io.druid.timeline.partition.SingleElementPartitionChunk;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Interval;
import org.joda.time.Period;
import javax.annotation.Nullable;
import javax.validation.constraints.NotNull;
import java.io.File;
import java.io.FilenameFilter;
import java.io.IOException;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
/**
*/
public class RealtimePlumberSchool implements PlumberSchool
{
private static final EmittingLogger log = new EmittingLogger(RealtimePlumberSchool.class);
private static final int defaultPending = 2;
private final Period windowPeriod;
private final File basePersistDirectory;
private final IndexGranularity segmentGranularity;
private final Object handoffCondition = new Object();
private volatile boolean shuttingDown = false;
private final int maxPendingPersists;
@JacksonInject
@NotNull
private volatile ServiceEmitter emitter;
@JacksonInject
@NotNull
private volatile QueryRunnerFactoryConglomerate conglomerate = null;
@JacksonInject
@NotNull
private volatile DataSegmentPusher dataSegmentPusher = null;
@JacksonInject
@NotNull
private volatile DataSegmentAnnouncer segmentAnnouncer = null;
@JacksonInject
@NotNull
private volatile SegmentPublisher segmentPublisher = null;
@JacksonInject
@NotNull
private volatile ServerView serverView = null;
@JacksonInject
@NotNull
@Processing
private volatile ExecutorService queryExecutorService = null;
private volatile VersioningPolicy versioningPolicy = null;
private volatile RejectionPolicyFactory rejectionPolicyFactory = null;
@ -136,7 +81,8 @@ public class RealtimePlumberSchool implements PlumberSchool
public RealtimePlumberSchool(
@JsonProperty("windowPeriod") Period windowPeriod,
@JsonProperty("basePersistDirectory") File basePersistDirectory,
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity,
@JsonProperty("maxPendingPersists") int maxPendingPersists
)
{
this.windowPeriod = windowPeriod;
@ -144,7 +90,9 @@ public class RealtimePlumberSchool implements PlumberSchool
this.segmentGranularity = segmentGranularity;
this.versioningPolicy = new IntervalStartVersioningPolicy();
this.rejectionPolicyFactory = new ServerTimeRejectionPolicyFactory();
this.maxPendingPersists = (maxPendingPersists > 0) ? maxPendingPersists : defaultPending;
Preconditions.checkArgument(maxPendingPersists <= 0, "RealtimePlumberSchool requires maxPendingPersists > 0");
Preconditions.checkNotNull(windowPeriod, "RealtimePlumberSchool requires a windowPeriod.");
Preconditions.checkNotNull(basePersistDirectory, "RealtimePlumberSchool requires a basePersistDirectory.");
Preconditions.checkNotNull(segmentGranularity, "RealtimePlumberSchool requires a segmentGranularity.");
@ -205,573 +153,24 @@ public class RealtimePlumberSchool implements PlumberSchool
final RejectionPolicy rejectionPolicy = rejectionPolicyFactory.create(windowPeriod);
log.info("Creating plumber using rejectionPolicy[%s]", rejectionPolicy);
return new Plumber()
{
private volatile boolean stopped = false;
private volatile ExecutorService persistExecutor = null;
private volatile ScheduledExecutorService scheduledExecutor = null;
private final Map<Long, Sink> sinks = Maps.newConcurrentMap();
private final VersionedIntervalTimeline<String, Sink> sinkTimeline = new VersionedIntervalTimeline<String, Sink>(
String.CASE_INSENSITIVE_ORDER
);
@Override
public void startJob()
{
computeBaseDir(schema).mkdirs();
initializeExecutors();
bootstrapSinksFromDisk();
registerServerViewCallback();
startPersistThread();
}
@Override
public Sink getSink(long timestamp)
{
if (!rejectionPolicy.accept(timestamp)) {
return null;
}
final long truncatedTime = segmentGranularity.truncate(timestamp);
Sink retVal = sinks.get(truncatedTime);
if (retVal == null) {
final Interval sinkInterval = new Interval(
new DateTime(truncatedTime),
segmentGranularity.increment(new DateTime(truncatedTime))
);
retVal = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval));
try {
segmentAnnouncer.announceSegment(retVal.getSegment());
sinks.put(truncatedTime, retVal);
sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), new SingleElementPartitionChunk<Sink>(retVal));
}
catch (IOException e) {
log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource())
.addData("interval", retVal.getInterval())
.emit();
}
}
return retVal;
}
@Override
public <T> QueryRunner<T> getQueryRunner(final Query<T> query)
{
final QueryRunnerFactory<T, Query<T>> factory = conglomerate.findFactory(query);
final QueryToolChest<T, Query<T>> toolchest = factory.getToolchest();
final Function<Query<T>, ServiceMetricEvent.Builder> builderFn =
new Function<Query<T>, ServiceMetricEvent.Builder>()
{
@Override
public ServiceMetricEvent.Builder apply(@Nullable Query<T> input)
{
return toolchest.makeMetricBuilder(query);
}
};
List<TimelineObjectHolder<String, Sink>> querySinks = Lists.newArrayList();
for (Interval interval : query.getIntervals()) {
querySinks.addAll(sinkTimeline.lookup(interval));
}
return toolchest.mergeResults(
factory.mergeRunners(
queryExecutorService,
FunctionalIterable
.create(querySinks)
.transform(
new Function<TimelineObjectHolder<String, Sink>, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(TimelineObjectHolder<String, Sink> holder)
{
final Sink theSink = holder.getObject().getChunk(0).getObject();
return new SpecificSegmentQueryRunner<T>(
new MetricsEmittingQueryRunner<T>(
return new RealtimePlumber(
windowPeriod,
basePersistDirectory,
segmentGranularity,
schema,
metrics,
rejectionPolicy,
emitter,
builderFn,
factory.mergeRunners(
MoreExecutors.sameThreadExecutor(),
Iterables.transform(
theSink,
new Function<FireHydrant, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(FireHydrant input)
{
return factory.createRunner(input.getSegment());
}
}
)
)
),
new SpecificSegmentSpec(
new SegmentDescriptor(
holder.getInterval(),
theSink.getSegment().getVersion(),
theSink.getSegment().getShardSpec().getPartitionNum()
)
)
conglomerate,
segmentAnnouncer,
queryExecutorService,
versioningPolicy,
dataSegmentPusher,
segmentPublisher,
serverView,
maxPendingPersists
);
}
}
)
)
);
}
@Override
public void persist(final Runnable commitRunnable)
{
final List<Pair<FireHydrant, Interval>> indexesToPersist = Lists.newArrayList();
for (Sink sink : sinks.values()) {
if (sink.swappable()) {
indexesToPersist.add(Pair.of(sink.swap(), sink.getInterval()));
}
}
log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource());
persistExecutor.execute(
new ThreadRenamingRunnable(String.format("%s-incremental-persist", schema.getDataSource()))
{
@Override
public void doRun()
{
for (Pair<FireHydrant, Interval> pair : indexesToPersist) {
metrics.incrementRowOutputCount(persistHydrant(pair.lhs, schema, pair.rhs));
}
commitRunnable.run();
}
}
);
}
// Submits persist-n-merge task for a Sink to the persistExecutor
private void persistAndMerge(final long truncatedTime, final Sink sink)
{
final String threadName = String.format(
"%s-%s-persist-n-merge", schema.getDataSource(), new DateTime(truncatedTime)
);
persistExecutor.execute(
new ThreadRenamingRunnable(threadName)
{
@Override
public void doRun()
{
final Interval interval = sink.getInterval();
for (FireHydrant hydrant : sink) {
if (!hydrant.hasSwapped()) {
log.info("Hydrant[%s] hasn't swapped yet, swapping. Sink[%s]", hydrant, sink);
final int rowCount = persistHydrant(hydrant, schema, interval);
metrics.incrementRowOutputCount(rowCount);
}
}
final File mergedTarget = new File(computePersistDir(schema, interval), "merged");
if (mergedTarget.exists()) {
log.info("Skipping already-merged sink: %s", sink);
return;
}
File mergedFile = null;
try {
List<QueryableIndex> indexes = Lists.newArrayList();
for (FireHydrant fireHydrant : sink) {
Segment segment = fireHydrant.getSegment();
final QueryableIndex queryableIndex = segment.asQueryableIndex();
log.info("Adding hydrant[%s]", fireHydrant);
indexes.add(queryableIndex);
}
mergedFile = IndexMerger.mergeQueryableIndex(
indexes,
schema.getAggregators(),
mergedTarget
);
QueryableIndex index = IndexIO.loadIndex(mergedFile);
DataSegment segment = dataSegmentPusher.push(
mergedFile,
sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions()))
);
segmentPublisher.publishSegment(segment);
}
catch (IOException e) {
log.makeAlert(e, "Failed to persist merged index[%s]", schema.getDataSource())
.addData("interval", interval)
.emit();
if (shuttingDown) {
// We're trying to shut down, and this segment failed to push. Let's just get rid of it.
abandonSegment(truncatedTime, sink);
}
}
if (mergedFile != null) {
try {
log.info("Deleting Index File[%s]", mergedFile);
FileUtils.deleteDirectory(mergedFile);
}
catch (IOException e) {
log.warn(e, "Error deleting directory[%s]", mergedFile);
}
}
}
}
);
}
@Override
public void finishJob()
{
log.info("Shutting down...");
shuttingDown = true;
for (final Map.Entry<Long, Sink> entry : sinks.entrySet()) {
persistAndMerge(entry.getKey(), entry.getValue());
}
while (!sinks.isEmpty()) {
try {
log.info(
"Cannot shut down yet! Sinks remaining: %s",
Joiner.on(", ").join(
Iterables.transform(
sinks.values(),
new Function<Sink, String>()
{
@Override
public String apply(Sink input)
{
return input.getSegment().getIdentifier();
}
}
)
)
);
synchronized (handoffCondition) {
while (!sinks.isEmpty()) {
handoffCondition.wait();
}
}
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
}
}
// scheduledExecutor is shutdown here, but persistExecutor is shutdown when the
// ServerView sends it a new segment callback
if (scheduledExecutor != null) {
scheduledExecutor.shutdown();
}
stopped = true;
}
private void initializeExecutors()
{
if (persistExecutor == null) {
persistExecutor = Executors.newFixedThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("plumber_persist_%d")
.build()
);
}
if (scheduledExecutor == null) {
scheduledExecutor = Executors.newScheduledThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("plumber_scheduled_%d")
.build()
);
}
}
private void bootstrapSinksFromDisk()
{
File baseDir = computeBaseDir(schema);
if (baseDir == null || !baseDir.exists()) {
return;
}
File[] files = baseDir.listFiles();
if (files == null) {
return;
}
for (File sinkDir : files) {
Interval sinkInterval = new Interval(sinkDir.getName().replace("_", "/"));
//final File[] sinkFiles = sinkDir.listFiles();
// To avoid reading and listing of "merged" dir
final File[] sinkFiles = sinkDir.listFiles(
new FilenameFilter()
{
@Override
public boolean accept(File dir, String fileName)
{
return !(Ints.tryParse(fileName) == null);
}
}
);
Arrays.sort(
sinkFiles,
new Comparator<File>()
{
@Override
public int compare(File o1, File o2)
{
try {
return Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName()));
}
catch (NumberFormatException e) {
log.error(e, "Couldn't compare as numbers? [%s][%s]", o1, o2);
return o1.compareTo(o2);
}
}
}
);
try {
List<FireHydrant> hydrants = Lists.newArrayList();
for (File segmentDir : sinkFiles) {
log.info("Loading previously persisted segment at [%s]", segmentDir);
// Although this has been tackled at start of this method.
// Just a doubly-check added to skip "merged" dir. from being added to hydrants
// If 100% sure that this is not needed, this check can be removed.
if (Ints.tryParse(segmentDir.getName()) == null) {
continue;
}
hydrants.add(
new FireHydrant(
new QueryableIndexSegment(null, IndexIO.loadIndex(segmentDir)),
Integer.parseInt(segmentDir.getName())
)
);
}
Sink currSink = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval), hydrants);
sinks.put(sinkInterval.getStartMillis(), currSink);
sinkTimeline.add(
currSink.getInterval(),
currSink.getVersion(),
new SingleElementPartitionChunk<Sink>(currSink)
);
segmentAnnouncer.announceSegment(currSink.getSegment());
}
catch (IOException e) {
log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource())
.addData("interval", sinkInterval)
.emit();
}
}
}
private void registerServerViewCallback()
{
serverView.registerSegmentCallback(
persistExecutor,
new ServerView.BaseSegmentCallback()
{
@Override
public ServerView.CallbackAction segmentAdded(DruidServer server, DataSegment segment)
{
if (stopped) {
log.info("Unregistering ServerViewCallback");
persistExecutor.shutdown();
return ServerView.CallbackAction.UNREGISTER;
}
if ("realtime".equals(server.getType())) {
return ServerView.CallbackAction.CONTINUE;
}
log.debug("Checking segment[%s] on server[%s]", segment, server);
if (schema.getDataSource().equals(segment.getDataSource())) {
final Interval interval = segment.getInterval();
for (Map.Entry<Long, Sink> entry : sinks.entrySet()) {
final Long sinkKey = entry.getKey();
if (interval.contains(sinkKey)) {
final Sink sink = entry.getValue();
log.info("Segment[%s] matches sink[%s] on server[%s]", segment, sink, server);
final String segmentVersion = segment.getVersion();
final String sinkVersion = sink.getSegment().getVersion();
if (segmentVersion.compareTo(sinkVersion) >= 0) {
log.info("Segment version[%s] >= sink version[%s]", segmentVersion, sinkVersion);
abandonSegment(sinkKey, sink);
}
}
}
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
}
private void startPersistThread()
{
final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis();
final long windowMillis = windowPeriod.toStandardDuration().getMillis();
log.info(
"Expect to run at [%s]",
new DateTime().plus(
new Duration(System.currentTimeMillis(), segmentGranularity.increment(truncatedNow) + windowMillis)
)
);
ScheduledExecutors
.scheduleAtFixedRate(
scheduledExecutor,
new Duration(System.currentTimeMillis(), segmentGranularity.increment(truncatedNow) + windowMillis),
new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)),
new ThreadRenamingCallable<ScheduledExecutors.Signal>(
String.format(
"%s-overseer-%d",
schema.getDataSource(),
schema.getShardSpec().getPartitionNum()
)
)
{
@Override
public ScheduledExecutors.Signal doCall()
{
if (stopped) {
log.info("Stopping merge-n-push overseer thread");
return ScheduledExecutors.Signal.STOP;
}
log.info("Starting merge and push.");
long minTimestamp = segmentGranularity.truncate(
rejectionPolicy.getCurrMaxTime().minus(windowMillis)
).getMillis();
List<Map.Entry<Long, Sink>> sinksToPush = Lists.newArrayList();
for (Map.Entry<Long, Sink> entry : sinks.entrySet()) {
final Long intervalStart = entry.getKey();
if (intervalStart < minTimestamp) {
log.info("Adding entry[%s] for merge and push.", entry);
sinksToPush.add(entry);
}
}
for (final Map.Entry<Long, Sink> entry : sinksToPush) {
persistAndMerge(entry.getKey(), entry.getValue());
}
if (stopped) {
log.info("Stopping merge-n-push overseer thread");
return ScheduledExecutors.Signal.STOP;
} else {
return ScheduledExecutors.Signal.REPEAT;
}
}
}
);
}
/**
* Unannounces a given sink and removes all local references to it.
*/
private void abandonSegment(final long truncatedTime, final Sink sink) {
try {
segmentAnnouncer.unannounceSegment(sink.getSegment());
FileUtils.deleteDirectory(computePersistDir(schema, sink.getInterval()));
log.info("Removing sinkKey %d for segment %s", truncatedTime, sink.getSegment().getIdentifier());
sinks.remove(truncatedTime);
sinkTimeline.remove(
sink.getInterval(),
sink.getVersion(),
new SingleElementPartitionChunk<>(sink)
);
synchronized (handoffCondition) {
handoffCondition.notifyAll();
}
}
catch (Exception e) {
log.makeAlert(e, "Unable to abandon old segment for dataSource[%s]", schema.getDataSource())
.addData("interval", sink.getInterval())
.emit();
}
}
};
}
private File computeBaseDir(Schema schema)
{
return new File(basePersistDirectory, schema.getDataSource());
}
private File computePersistDir(Schema schema, Interval interval)
{
return new File(computeBaseDir(schema), interval.toString().replace("/", "_"));
}
/**
* Persists the given hydrant and returns the number of rows persisted
*
* @param indexToPersist
* @param schema
* @param interval
*
* @return the number of rows persisted
*/
private int persistHydrant(FireHydrant indexToPersist, Schema schema, Interval interval)
{
if (indexToPersist.hasSwapped()) {
log.info(
"DataSource[%s], Interval[%s], Hydrant[%s] already swapped. Ignoring request to persist.",
schema.getDataSource(), interval, indexToPersist
);
return 0;
}
log.info("DataSource[%s], Interval[%s], persisting Hydrant[%s]", schema.getDataSource(), interval, indexToPersist);
try {
int numRows = indexToPersist.getIndex().size();
File persistedFile = IndexMerger.persist(
indexToPersist.getIndex(),
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount()))
);
indexToPersist.swapSegment(new QueryableIndexSegment(null, IndexIO.loadIndex(persistedFile)));
return numRows;
}
catch (IOException e) {
log.makeAlert("dataSource[%s] -- incremental persist failed", schema.getDataSource())
.addData("interval", interval)
.addData("count", indexToPersist.getCount())
.emit();
throw Throwables.propagate(e);
}
}
private void verifyState()
{

View File

@ -178,12 +178,12 @@ public class Sink implements Iterable<FireHydrant>
FireHydrant old;
if (currIndex == null) { // Only happens on initialization, cannot synchronize on null
old = currIndex;
currIndex = new FireHydrant(newIndex, hydrants.size());
currIndex = new FireHydrant(newIndex, hydrants.size(), getSegment().getIdentifier());
hydrants.add(currIndex);
} else {
synchronized (currIndex) {
old = currIndex;
currIndex = new FireHydrant(newIndex, hydrants.size());
currIndex = new FireHydrant(newIndex, hydrants.size(), getSegment().getIdentifier());
hydrants.add(currIndex);
}
}

View File

@ -0,0 +1,36 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.bridge;
import com.google.inject.BindingAnnotation;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
*/
@BindingAnnotation
@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
@Retention(RetentionPolicy.RUNTIME)
public @interface Bridge
{
}

View File

@ -0,0 +1,31 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.bridge;
import io.druid.curator.CuratorConfig;
import org.skife.config.Config;
/**
*/
public abstract class BridgeCuratorConfig extends CuratorConfig
{
@Config("druid.bridge.zk.service.host")
public abstract String getParentZkHosts();
}

View File

@ -0,0 +1,126 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.bridge;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.inject.Inject;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.common.logger.Logger;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.StatusResponseHandler;
import com.metamx.http.client.response.StatusResponseHolder;
import io.druid.client.selector.Server;
import io.druid.curator.discovery.ServerDiscoverySelector;
import io.druid.guice.annotations.Global;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.SegmentDescriptor;
import org.joda.time.Interval;
import java.net.URL;
import java.util.List;
/**
*/
public class BridgeQuerySegmentWalker implements QuerySegmentWalker
{
private static final Logger log = new Logger(BridgeQuerySegmentWalker.class);
private final ServerDiscoverySelector brokerSelector;
private final HttpClient httpClient;
private final ObjectMapper jsonMapper;
private final StatusResponseHandler responseHandler;
@Inject
public BridgeQuerySegmentWalker(
ServerDiscoverySelector brokerSelector,
@Global HttpClient httpClient,
ObjectMapper jsonMapper
)
{
this.brokerSelector = brokerSelector;
this.httpClient = httpClient;
this.jsonMapper = jsonMapper;
this.responseHandler = new StatusResponseHandler(Charsets.UTF_8);
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(
Query<T> query, Iterable<Interval> intervals
)
{
return makeRunner();
}
@Override
public <T> QueryRunner<T> getQueryRunnerForSegments(
Query<T> query, Iterable<SegmentDescriptor> specs
)
{
return makeRunner();
}
private <T> QueryRunner<T> makeRunner()
{
return new QueryRunner<T>()
{
@Override
public Sequence<T> run(Query<T> query)
{
try {
Server instance = brokerSelector.pick();
if (instance == null) {
return Sequences.empty();
}
final String url = String.format(
"http://%s/druid/v2/",
brokerSelector.pick().getHost()
);
StatusResponseHolder response = httpClient.post(new URL(url))
.setContent(
"application/json",
jsonMapper.writeValueAsBytes(query)
)
.go(responseHandler)
.get();
List<T> results = jsonMapper.readValue(
response.getContent(), new TypeReference<List<T>>()
{
}
);
return Sequences.simple(results);
}
catch (Exception e) {
log.error(e, "Exception with bridge query");
return Sequences.empty();
}
}
};
}
}

View File

@ -0,0 +1,132 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.bridge;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import io.druid.client.DruidServer;
import io.druid.client.ServerView;
import io.druid.concurrent.Execs;
import io.druid.db.DatabaseSegmentManager;
import io.druid.segment.realtime.DbSegmentPublisher;
import io.druid.server.coordination.BaseZkCoordinator;
import io.druid.server.coordination.DataSegmentChangeCallback;
import io.druid.server.coordination.DataSegmentChangeHandler;
import io.druid.server.coordination.DruidServerMetadata;
import io.druid.server.initialization.ZkPathsConfig;
import io.druid.timeline.DataSegment;
import org.apache.curator.framework.CuratorFramework;
import java.util.concurrent.ExecutorService;
/**
*/
public class BridgeZkCoordinator extends BaseZkCoordinator
{
private static final Logger log = new Logger(BaseZkCoordinator.class);
private final DbSegmentPublisher dbSegmentPublisher;
private final DatabaseSegmentManager databaseSegmentManager;
private final ServerView serverView;
private final ExecutorService exec = Execs.singleThreaded("BridgeZkCoordinatorServerView-%s");
@Inject
public BridgeZkCoordinator(
ObjectMapper jsonMapper,
ZkPathsConfig zkPaths,
DruidServerMetadata me,
@Bridge CuratorFramework curator,
DbSegmentPublisher dbSegmentPublisher,
DatabaseSegmentManager databaseSegmentManager,
ServerView serverView
)
{
super(jsonMapper, zkPaths, me, curator);
this.dbSegmentPublisher = dbSegmentPublisher;
this.databaseSegmentManager = databaseSegmentManager;
this.serverView = serverView;
}
@Override
public void loadLocalCache()
{
// do nothing
}
@Override
public DataSegmentChangeHandler getDataSegmentChangeHandler()
{
return BridgeZkCoordinator.this;
}
@Override
public void addSegment(final DataSegment segment, final DataSegmentChangeCallback callback)
{
try {
log.info("Publishing segment %s", segment.getIdentifier());
dbSegmentPublisher.publishSegment(segment);
serverView.registerSegmentCallback(
exec,
new ServerView.BaseSegmentCallback()
{
@Override
public ServerView.CallbackAction segmentAdded(
DruidServer server, DataSegment theSegment
)
{
if (theSegment.equals(segment)) {
callback.execute();
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override
public void removeSegment(final DataSegment segment, final DataSegmentChangeCallback callback)
{
databaseSegmentManager.removeSegment(segment.getDataSource(), segment.getIdentifier());
serverView.registerSegmentCallback(
exec,
new ServerView.BaseSegmentCallback()
{
@Override
public ServerView.CallbackAction segmentRemoved(
DruidServer server, DataSegment theSegment
)
{
if (theSegment.equals(segment)) {
callback.execute();
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
}
}

View File

@ -0,0 +1,391 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.bridge;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.inject.Inject;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.emitter.EmittingLogger;
import io.druid.client.DruidServer;
import io.druid.client.ServerInventoryView;
import io.druid.client.ServerView;
import io.druid.concurrent.Execs;
import io.druid.curator.announcement.Announcer;
import io.druid.guice.ManageLifecycle;
import io.druid.guice.annotations.Self;
import io.druid.server.DruidNode;
import io.druid.server.coordination.AbstractDataSegmentAnnouncer;
import io.druid.server.coordination.DataSegmentAnnouncer;
import io.druid.server.coordination.DruidServerMetadata;
import io.druid.timeline.DataSegment;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.leader.LeaderLatch;
import org.apache.curator.framework.recipes.leader.LeaderLatchListener;
import org.apache.curator.utils.ZKPaths;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ManageLifecycle
public class DruidClusterBridge
{
public static final String BRIDGE_OWNER_NODE = "_BRIDGE";
public static final String NODE_TYPE = "bridge";
private static final EmittingLogger log = new EmittingLogger(DruidClusterBridge.class);
private final ObjectMapper jsonMapper;
private final DruidClusterBridgeConfig config;
private final ScheduledExecutorService exec;
private final DruidNode self;
// Communicates to the ZK cluster that this bridge node is deployed at
private final CuratorFramework curator;
private final AtomicReference<LeaderLatch> leaderLatch;
// Communicates to the remote (parent) ZK cluster
private final BridgeZkCoordinator bridgeZkCoordinator;
private final Announcer announcer;
private final ServerInventoryView<Object> serverInventoryView;
private final Map<DataSegment, Integer> segments = Maps.newHashMap();
private final Object lock = new Object();
private volatile boolean started = false;
private volatile boolean leader = false;
@Inject
public DruidClusterBridge(
ObjectMapper jsonMapper,
DruidClusterBridgeConfig config,
ScheduledExecutorFactory scheduledExecutorFactory,
@Self DruidNode self,
CuratorFramework curator,
AtomicReference<LeaderLatch> leaderLatch,
BridgeZkCoordinator bridgeZkCoordinator,
@Bridge Announcer announcer,
@Bridge final AbstractDataSegmentAnnouncer dataSegmentAnnouncer,
ServerInventoryView serverInventoryView
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.bridgeZkCoordinator = bridgeZkCoordinator;
this.announcer = announcer;
this.serverInventoryView = serverInventoryView;
this.curator = curator;
this.leaderLatch = leaderLatch;
this.exec = scheduledExecutorFactory.create(1, "Coordinator-Exec--%d");
this.self = self;
ExecutorService serverInventoryViewExec = Executors.newFixedThreadPool(
1,
new ThreadFactoryBuilder().setDaemon(true)
.setNameFormat("DruidClusterBridge-ServerInventoryView-%d")
.build()
);
serverInventoryView.registerSegmentCallback(
serverInventoryViewExec,
new ServerView.BaseSegmentCallback()
{
@Override
public ServerView.CallbackAction segmentAdded(
DruidServer server, DataSegment segment
)
{
try {
synchronized (lock) {
Integer count = segments.get(segment);
if (count == null) {
segments.put(segment, 1);
dataSegmentAnnouncer.announceSegment(segment);
} else {
segments.put(segment, count + 1);
}
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
return ServerView.CallbackAction.CONTINUE;
}
@Override
public ServerView.CallbackAction segmentRemoved(DruidServer server, DataSegment segment)
{
try {
synchronized (lock) {
serverRemovedSegment(dataSegmentAnnouncer, segment, server);
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
serverInventoryView.registerServerCallback(
serverInventoryViewExec,
new ServerView.ServerCallback()
{
@Override
public ServerView.CallbackAction serverRemoved(DruidServer server)
{
try {
for (DataSegment dataSegment : server.getSegments().values()) {
serverRemovedSegment(dataSegmentAnnouncer, dataSegment, server);
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
}
public boolean isLeader()
{
return leader;
}
@LifecycleStart
public void start()
{
synchronized (lock) {
if (started) {
return;
}
started = true;
createNewLeaderLatch();
try {
leaderLatch.get().start();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
private LeaderLatch createNewLeaderLatch()
{
final LeaderLatch newLeaderLatch = new LeaderLatch(
curator, ZKPaths.makePath(config.getConnectorPath(), BRIDGE_OWNER_NODE), self.getHost()
);
newLeaderLatch.addListener(
new LeaderLatchListener()
{
@Override
public void isLeader()
{
becomeLeader();
}
@Override
public void notLeader()
{
stopBeingLeader();
}
},
Execs.singleThreaded("CoordinatorLeader-%s")
);
return leaderLatch.getAndSet(newLeaderLatch);
}
@LifecycleStop
public void stop()
{
synchronized (lock) {
if (!started) {
return;
}
stopBeingLeader();
try {
leaderLatch.get().close();
}
catch (IOException e) {
log.warn(e, "Unable to close leaderLatch, ignoring");
}
exec.shutdownNow();
started = false;
}
}
private void becomeLeader()
{
synchronized (lock) {
if (!started) {
return;
}
log.info("Go-Go Gadgetmobile! Starting bridge in %s", config.getStartDelay());
try {
bridgeZkCoordinator.start();
serverInventoryView.start();
ScheduledExecutors.scheduleWithFixedDelay(
exec,
config.getStartDelay(),
config.getPeriod(),
new Callable<ScheduledExecutors.Signal>()
{
@Override
public ScheduledExecutors.Signal call()
{
if (leader) {
Iterable<DruidServer> servers = FunctionalIterable
.create(serverInventoryView.getInventory())
.filter(
new Predicate<DruidServer>()
{
@Override
public boolean apply(
DruidServer input
)
{
return !input.isRealtime();
}
}
);
long totalMaxSize = 0;
for (DruidServer server : servers) {
totalMaxSize += server.getMaxSize();
}
if (totalMaxSize == 0) {
log.warn("No servers founds!");
} else {
DruidServerMetadata me = new DruidServerMetadata(
self.getHost(),
self.getHost(),
totalMaxSize,
NODE_TYPE,
config.getTier(),
config.getPriority()
);
try {
final String path = ZKPaths.makePath(config.getAnnouncementsPath(), self.getHost());
log.info("Updating [%s] to have a maxSize of[%,d] bytes", self.getHost(), totalMaxSize);
announcer.update(path, jsonMapper.writeValueAsBytes(me));
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
if (leader) { // (We might no longer be leader)
return ScheduledExecutors.Signal.REPEAT;
} else {
return ScheduledExecutors.Signal.STOP;
}
}
}
);
leader = true;
}
catch (Exception e) {
log.makeAlert(e, "Exception becoming leader")
.emit();
final LeaderLatch oldLatch = createNewLeaderLatch();
Closeables.closeQuietly(oldLatch);
try {
leaderLatch.get().start();
}
catch (Exception e1) {
// If an exception gets thrown out here, then the bridge will zombie out 'cause it won't be looking for
// the latch anymore. I don't believe it's actually possible for an Exception to throw out here, but
// Curator likes to have "throws Exception" on methods so it might happen...
log.makeAlert(e1, "I am a zombie")
.emit();
}
}
}
}
private void stopBeingLeader()
{
synchronized (lock) {
try {
log.info("I'll get you next time, Gadget. Next time!");
bridgeZkCoordinator.stop();
serverInventoryView.stop();
leader = false;
}
catch (Exception e) {
log.makeAlert(e, "Unable to stopBeingLeader").emit();
}
}
}
private void serverRemovedSegment(DataSegmentAnnouncer dataSegmentAnnouncer, DataSegment segment, DruidServer server)
throws IOException
{
Integer count = segments.get(segment);
if (count != null) {
if (count == 1) {
dataSegmentAnnouncer.unannounceSegment(segment);
segments.remove(segment);
} else {
segments.put(segment, count - 1);
}
} else {
log.makeAlert("Trying to remove a segment that was never added?")
.addData("server", server.getHost())
.addData("segmentId", segment.getIdentifier())
.emit();
}
}
}

View File

@ -0,0 +1,52 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.bridge;
import io.druid.client.DruidServer;
import io.druid.server.initialization.ZkPathsConfig;
import org.joda.time.Duration;
import org.skife.config.Config;
import org.skife.config.Default;
/**
*/
public abstract class DruidClusterBridgeConfig extends ZkPathsConfig
{
@Config("druid.server.tier")
@Default(DruidServer.DEFAULT_TIER)
public abstract String getTier();
@Config("druid.bridge.startDelay")
@Default("PT300s")
public abstract Duration getStartDelay();
@Config("druid.bridge.period")
@Default("PT60s")
public abstract Duration getPeriod();
@Config("druid.bridge.broker.serviceName")
public abstract String getBrokerServiceName();
@Config("druid.server.priority")
public int getPriority()
{
return DruidServer.DEFAULT_PRIORITY;
}
}

View File

@ -0,0 +1,196 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.coordination;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.emitter.EmittingLogger;
import io.druid.server.initialization.ZkPathsConfig;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
import org.apache.curator.utils.ZKPaths;
import java.io.IOException;
/**
*/
public abstract class BaseZkCoordinator implements DataSegmentChangeHandler
{
private static final EmittingLogger log = new EmittingLogger(ZkCoordinator.class);
private final Object lock = new Object();
private final ObjectMapper jsonMapper;
private final ZkPathsConfig zkPaths;
private final DruidServerMetadata me;
private final CuratorFramework curator;
private volatile PathChildrenCache loadQueueCache;
private volatile boolean started;
public BaseZkCoordinator(
ObjectMapper jsonMapper,
ZkPathsConfig zkPaths,
DruidServerMetadata me,
CuratorFramework curator
)
{
this.jsonMapper = jsonMapper;
this.zkPaths = zkPaths;
this.me = me;
this.curator = curator;
}
@LifecycleStart
public void start() throws IOException
{
synchronized (lock) {
if (started) {
return;
}
log.info("Starting zkCoordinator for server[%s]", me);
final String loadQueueLocation = ZKPaths.makePath(zkPaths.getLoadQueuePath(), me.getName());
final String servedSegmentsLocation = ZKPaths.makePath(zkPaths.getServedSegmentsPath(), me.getName());
final String liveSegmentsLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), me.getName());
loadQueueCache = new PathChildrenCache(
curator,
loadQueueLocation,
true,
true,
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ZkCoordinator-%s").build()
);
try {
curator.newNamespaceAwareEnsurePath(loadQueueLocation).ensure(curator.getZookeeperClient());
curator.newNamespaceAwareEnsurePath(servedSegmentsLocation).ensure(curator.getZookeeperClient());
curator.newNamespaceAwareEnsurePath(liveSegmentsLocation).ensure(curator.getZookeeperClient());
loadLocalCache();
loadQueueCache.getListenable().addListener(
new PathChildrenCacheListener()
{
@Override
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
{
final ChildData child = event.getData();
switch (event.getType()) {
case CHILD_ADDED:
final String path = child.getPath();
final DataSegmentChangeRequest segment = jsonMapper.readValue(
child.getData(), DataSegmentChangeRequest.class
);
log.info("New node[%s] with segmentClass[%s]", path, segment.getClass());
try {
segment.go(
getDataSegmentChangeHandler(),
new DataSegmentChangeCallback()
{
boolean hasRun = false;
@Override
public void execute()
{
try {
if (!hasRun) {
curator.delete().guaranteed().forPath(path);
log.info("Completed processing for node[%s]", path);
hasRun = true;
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
);
}
catch (Exception e) {
try {
curator.delete().guaranteed().forPath(path);
}
catch (Exception e1) {
log.info(e1, "Failed to delete node[%s], but ignoring exception.", path);
}
log.makeAlert(e, "Segment load/unload: uncaught exception.")
.addData("node", path)
.addData("nodeProperties", segment)
.emit();
}
break;
case CHILD_REMOVED:
log.info("%s was removed", event.getData().getPath());
break;
default:
log.info("Ignoring event[%s]", event);
}
}
}
);
loadQueueCache.start();
}
catch (Exception e) {
Throwables.propagateIfPossible(e, IOException.class);
throw Throwables.propagate(e);
}
started = true;
}
}
@LifecycleStop
public void stop()
{
log.info("Stopping ZkCoordinator for [%s]", me);
synchronized (lock) {
if (!started) {
return;
}
try {
loadQueueCache.close();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
finally {
loadQueueCache = null;
started = false;
}
}
}
public abstract void loadLocalCache();
public abstract DataSegmentChangeHandler getDataSegmentChangeHandler();
}

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