merge changes from druid-0.7.x

This commit is contained in:
nishantmonu51 2014-07-31 16:07:22 +05:30
commit 637bd35785
206 changed files with 10435 additions and 688 deletions

View File

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

View File

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

View File

@ -24,6 +24,7 @@ import com.google.common.base.Throwables;
import com.google.common.collect.Ordering;
import com.google.common.io.Closeables;
import com.metamx.common.guava.Accumulator;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Yielder;
import com.metamx.common.guava.Yielders;
@ -70,7 +71,7 @@ public class OrderedMergeSequence<T> implements Sequence<T>
return yielder.get();
}
finally {
Closeables.closeQuietly(yielder);
CloseQuietly.close(yielder);
}
}

View File

@ -46,14 +46,9 @@ public class SerializerUtils
public void writeString(OutputSupplier<? extends OutputStream> supplier, String name) throws IOException
{
OutputStream out = null;
try {
out = supplier.getOutput();
try (OutputStream out = supplier.getOutput()) {
writeString(out, name);
}
finally {
Closeables.closeQuietly(out);
}
}
public void writeString(WritableByteChannel out, String name) throws IOException

View File

@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server;
package io.druid.common.utils;
import java.lang.reflect.InvocationTargetException;

View File

@ -155,8 +155,8 @@ Determine the number of distinct are assigned to.
### HyperUnique aggregator
Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to compute the estimated cardinality of a dimension that has been aggregated as a hyperUnique metric at indexing time.
Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to compute the estimated cardinality of a dimension that has been aggregated as a "hyperUnique" metric at indexing time.
```json
{ "type" : "hyperUnique", "name" : <output_name>, "fieldName" : <metric_name> }
```
```

View File

@ -0,0 +1,90 @@
### ApproxHistogram aggregator
This aggregator is based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf) to compute approximate histograms.
To use this feature, an "approxHistogram" aggregator must be included at indexing time. The ingestion aggregator can only apply to numeric values. To query for results, an "approxHistogramFold" aggregator must be included in the query.
```json
{
"type" : "approxHistogram(ingestion), approxHistogramFold(query)",
"name" : <output_name>,
"fieldName" : <metric_name>,
"resolution" : <integer>,
"numBuckets" : <integer>,
"lowerLimit" : <float>,
"upperLimit" : <float>
}
```
|Property|Description|Default|
|--------|-----------|-------|
|`resolution`|Number of centroids (data points) to store. The higher the resolution, the more accurate results are, but the slower computation will be.|50|
|`numBuckets`|Number of output buckets for the resulting histogram.|7|
|`lowerLimit`/`upperLimit`|Restrict the approximation to the given range. The values outside this range will be aggregated into two centroids. Counts of values outside this range are still maintained. |-INF/+INF|
### Approximate Histogram post-aggregators
Post-aggregators used to transform opaque approximate histogram objects
into actual histogram representations, and to compute various distribution metrics.
#### equal buckets post-aggregator
Computes a visual representation of the approximate histogram with a given number of equal-sized bins
```json
{ "type" : "equalBuckets", "name" : <output_name>, "fieldName" : <aggregator_name>,
"numBuckets" : <count> }
```
#### buckets post-aggregator
Computes a visual representation given an initial breakpoint, offset, and a bucket size.
```json
{ "type" : "buckets", "name" : <output_name>, "fieldName" : <aggregator_name>,
"bucketSize" : <bucket_size>, "offset" : <offset> }
```
#### custom buckets post-aggregator
Computes a visual representation of the approximate histogram with bins laid out according to the given breaks
```json
{ "type" : "customBuckets", "name" : <output_name>, "fieldName" : <aggregator_name>,
"breaks" : [ <value>, <value>, ... ] }
```
#### min post-aggregator
Returns the minimum value of the underlying approximate histogram aggregator
```json
{ "type" : "min", "name" : <output_name>, "fieldName" : <aggregator_name> }
```
#### max post-aggregator
Returns the maximum value of the underlying approximate histogram aggregator
```json
{ "type" : "max", "name" : <output_name>, "fieldName" : <aggregator_name> }
```
#### quantile post-aggregator
Computes a single quantile based on the underlying approximate histogram aggregator
```json
{ "type" : "quantile", "name" : <output_name>, "fieldName" : <aggregator_name>,
"probability" : <quantile> }
```
#### quantiles post-aggregator
Computes an array of quantiles based on the underlying approximate histogram aggregator
```json
{ "type" : "quantiles", "name" : <output_name>, "fieldName" : <aggregator_name>,
"probabilities" : [ <quantile>, <quantile>, ... ] }
```

View File

@ -89,6 +89,9 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim
"user": "username",
"password": "passmeup",
"segmentTable": "segments"
},
"jobProperties": {
"mapreduce.job.queuename": "default"
}
}
```
@ -109,6 +112,7 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim
|leaveIntermediate|leave behind files in the workingPath when job completes or fails (debugging tool).|no|
|partitionsSpec|a specification of how to partition each time bucket into segments, absence of this property means no partitioning will occur.|no|
|metadataUpdateSpec|a specification of how to update the metadata for the druid cluster these segments belong to.|yes|
|jobProperties|a map of properties to add to the Hadoop job configuration.|no|
### Path specification

View File

@ -17,10 +17,7 @@ You can provision individual servers, loading Druid onto each machine (or buildi
You'll need an AWS account, S3 Bucket and an EC2 key pair from that account so that Whirr can connect to the cloud via the EC2 API. If you haven't generated a key pair, see the [AWS documentation](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html) or see this [Whirr FAQ](http://whirr.apache.org/faq.html#how-do-i-find-my-cloud-credentials).
### Installing Whirr
You must use a version of Whirr that includes and supports a Druid recipe. You can do it so in one of two ways:
#### Build the Following Version of Whirr
### Install Whirr
Clone the code from [https://github.com/druid-io/whirr](https://github.com/druid-io/whirr) and build Whirr:
git clone git@github.com:druid-io/whirr.git
@ -28,23 +25,23 @@ Clone the code from [https://github.com/druid-io/whirr](https://github.com/druid
git checkout trunk
mvn clean install -Dmaven.test.failure.ignore=true
#### Build the Latest Version of Whirr
Clone the code from the Whirr repository:
In order to run the test below, you'll also need two files that available only from a [standard install of Druid](http://druid.io/downloads.html) or the [Druid repo](https://github.com/metamx/druid/tree/master/examples/bin/examples):
* `druid/examples/bin/examples/wikipedia/wikipedia_realtime.spec`
* `druid/examples/bin/examples/indexing/wikipedia_realtime_task.json`
git clone git://git.apache.org/whirr.git
Then run `mvn install` from the root directory.
### Configure Whirr
The Whirr recipe for Druid is the configuration file `$WHIRR_HOME/recipies/druid.properties`. You can edit this file to suit your needs -- it is annotated and self-explanatory. Here are some hints about that file:
The Whirr recipe for Druid is the configuration file `$WHIRR_HOME/recipies/druid.properties`. You can edit this file to suit your needs; it is annotated and self-explanatory. Here are some hints about that file:
* Set `whirr.location-id` to a specific AWS region (e.g., us-east-1) if desired, else one will be chosen for you.
* Set `whirr.location-id` to a specific AWS region if desired. If this is left blank, a region is chosen for you. The default value is `us-east-1`.
* You can choose the hardware used with `whirr.hardware-id` to a specific instance type (e.g., m1.large). By default druid.properties, m3.2xlarge (broker, historical, middle manager), m1.xlarge (coordinator, overlord), and m1.small (zookeeper, mysql) are used.
* If you don't choose an image via `whirr.image-id` (image must be compatible with hardware), you'll get plain vanilla Linux. Default druid.properties uses ami-018c9568 (Ubuntu 12.04).
* SSH keys (not password protected) must exist for the local user. If they are in the default locations, `${sys:user.home}/.ssh/id_rsa` and `${sys:user.home}/.ssh/id_rsa.pub`, Whirr will find them. Otherwise, you'll have to specify them with `whirr.private-key-file` and `whirr.public-key-file`.
* Be sure to specify the absolute path of the Druid realtime spec file `realtime.spec` in `whirr.druid.realtime.spec.path`.
* Also make sure to specify the correct S3 bucket. Otherwise the cluster won't be able to process tasks.
* Two Druid cluster templates (see `whirr.instance-templates`) are provided: a small cluster running on a single EC2 instance, and a larger cluster running on multiple instances.
* You must specify the path to an S3 bucket. Otherwise the cluster won't be able to process tasks.
* To successfully submit the test task below, you'll need to specify the location of the `wikipedia_realtime.spec` in the property `whirr.druid.realtime.spec.path`.
* Specify Druid version only if [Druid extenions](Modules.html) are being used.
The following AWS information must be set in `druid.properties`, as environment variables, or in the file `$WHIRR_HOME/conf/credentials`:
@ -54,7 +51,7 @@ The following AWS information must be set in `druid.properties`, as environment
How to get the IDENTITY and CREDENTIAL keys is discussed above.
In order to configure each node, you can edit `services/druid/src/main/resources/functions/start_druid.sh` for JVM configuration and `services/druid/src/main/resources/functions/configure_[NODE_NAME].sh` for specific node configuration. For more information on configuration, read the Druid documentations about it (http://druid.io/docs/0.6.116/Configuration.html).
In order to configure each node, you can edit `services/druid/src/main/resources/functions/start_druid.sh` for JVM configuration and `services/druid/src/main/resources/functions/configure_[NODE_NAME].sh` for specific node configuration. For more information on configuration, see the [Druid configuration documentation](Configuration.html).
### Start a Test Cluster With Whirr
Run the following command:
@ -66,14 +63,14 @@ If Whirr starts without any errors, you should see the following message:
Running on provider aws-ec2 using identity <your-aws-id-here>
You can then use the EC2 dashboard to locate the instance and confirm that it has started up.
You can then use the EC2 dashboard to locate the instances and confirm that they have started up.
If both the instance and the Druid cluster launch successfully, a few minutes later other messages to STDOUT should follow with information returned from EC2, including the instance ID:
If both the instances and the Druid cluster launch successfully, a few minutes later other messages to STDOUT should follow with information returned from EC2, including the instance ID:
Started cluster of 1 instances
Started cluster of 8 instances
Cluster{instances=[Instance{roles=[zookeeper, druid-mysql, druid-coordinator, druid-broker, druid-historical, druid-realtime], publicIp= ...
The final message will contain login information for the instance.
The final message will contain login information for the instances.
Note that Whirr will return an exception if any of the nodes fail to launch, and the cluster will be destroyed. To destroy the cluster manually, run the following command:
@ -85,9 +82,11 @@ Note that Whirr will return an exception if any of the nodes fail to launch, and
Now you can run an indexing task and a simple query to see if all the nodes have launched correctly. We are going to use a Wikipedia example again. For a realtime indexing task, run the following command:
```bash
curl -X 'POST' -H 'Content-Type:application/json' -d @#{YOUR_DRUID_DIRECTORY}/examples/indexing/wikipedia_realtime_task.json #{OVERLORD_PUBLIC_IP_ADDR}:#{PORT}/druid/indexer/v1/task
curl -X 'POST' -H 'Content-Type:application/json' -d @#{PATH_TO}/wikipedia_realtime_task.json #{OVERLORD_PUBLIC_IP_ADDR}:#{PORT}/druid/indexer/v1/task
```
Issuing the request should return a task ID.
where OVERLORD_PUBLIC_IP_ADDR should be available from the EC2 information logged to STDOUT, the Overlord port is 8080 by default, and `wikipedia_realtime_task.json` is discussed above.
Issuing this request should return a task ID.
To check the state of the overlord, open up your browser and go to `#{OVERLORD_PUBLIC_IP_ADDR}:#{PORT}/console.html`.

View File

@ -154,6 +154,7 @@ Druid storage nodes maintain information about segments they have already downlo
|--------|-----------|-------|
|`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. | none (no caching) |
|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true|
|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|5 minutes|
|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir|
### Jetty Server Module

View File

@ -69,10 +69,9 @@ A sample worker setup spec is shown below:
"keyName":"keyName"
},
"userData":{
"classType":"galaxy",
"env":"druid",
"version":"druid_version",
"type":"sample_cluster/worker"
"impl":"string",
"data":"version=:VERSION:",
"versionReplacementString":":VERSION:"
}
}
```
@ -81,8 +80,8 @@ Issuing a GET request at the same URL will return the current worker setup spec
|Property|Description|Default|
|--------|-----------|-------|
|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be the same as the coordinator version.|none|
|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be druid.indexer.runner.minWorkerVersion.|none|
|`minNumWorkers`|The minimum number of workers that can be in the cluster at any given time.|0|
|`maxNumWorkers`|The maximum number of workers that can be in the cluster at any given time.|0|
|`nodeData`|A JSON object that contains metadata about new nodes to create.|none|
|`userData`|A JSON object that contains metadata about how the node should register itself on startup. This data is sent with node creation requests.|none|
|`nodeData`|A JSON object that describes how to launch new nodes. Currently, only EC2 is supported.|none; required|
|`userData`|A JSON object that describes how to configure new nodes. Currently, only EC2 is supported. If you have set druid.indexer.autoscale.workerVersion, this must have a versionReplacementString. Otherwise, a versionReplacementString is not necessary.|none; optional|

View File

@ -52,6 +52,6 @@ Middle managers pass their configurations down to their child peons. The middle
|`druid.indexer.runner.javaCommand`|Command required to execute java.|java|
|`druid.indexer.runner.javaOpts`|-X Java options to run the peon in its own JVM.|""|
|`druid.indexer.runner.classpath`|Java classpath for the peon.|System.getProperty("java.class.path")|
|`druid.indexer.runner.startPort`|The port that peons begin running on.|8080|
|`druid.indexer.runner.startPort`|The port that peons begin running on.|8081|
|`druid.indexer.runner.allowedPrefixes`|Whitelist of prefixes for configs that can be passed down to child peons.|"com.metamx", "druid", "io.druid", "user.timezone","file.encoding"|

View File

@ -22,6 +22,7 @@ Additional peon configs include:
|`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|/tmp/persistent/tasks|
|`druid.indexer.task.hadoopWorkingPath`|Temporary working directory for Hadoop tasks.|/tmp/druid-indexing|
|`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|50000|
|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.3.0|
|`druid.indexer.task.chathandler.type`|Choices are "noop" and "announce". Certain tasks will use service discovery to announce an HTTP endpoint that events can be posted to.|noop|
If the peon is running in remote mode, there must be an overlord up and running. Running peons in remote mode require the following configurations:

View File

@ -64,7 +64,7 @@ Example JavaScript aggregator:
"function": "function(delta, total) { return 100 * Math.abs(delta) / total; }"
}
```
### `hyperUniqueCardinality` post-aggregator
### HyperUnique Cardinality post-aggregator
The hyperUniqueCardinality post aggregator is used to wrap a hyperUnique object such that it can be used in post aggregations.
@ -90,8 +90,7 @@ It can be used in a sample calculation as so:
}
```
### Example Usage
#### Example Usage
In this example, lets calculate a simple percentage using post aggregators. Lets imagine our data set has a metric called "total".
@ -122,5 +121,4 @@ The format of the query JSON is as follows:
}
...
}
```
```

View File

@ -153,7 +153,7 @@ druid.indexer.logs.s3Bucket=#{LOGS_BUCKET}
druid.indexer.logs.s3Prefix=prod/logs/v1
# Dedicate more resources to peons
druid.indexer.runner.javaOpts=-server -Xmx6g -Xms6g -XX:NewSize=256m -XX:MaxNewSize=256m -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
druid.indexer.runner.javaOpts=-server -Xmx3g -XX:+UseG1GC -XX:MaxGCPauseMillis=100 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
druid.indexer.runner.taskDir=/mnt/persistent/task/
druid.indexer.task.taskDir=/mnt/persistent/task/
druid.indexer.task.chathandler.type=announce

View File

@ -37,6 +37,7 @@ Configuration:
-Ddruid.selectors.indexing.serviceName=overlord
-Ddruid.indexer.queue.startDelay=PT0M
-Ddruid.indexer.runner.javaOpts="-server -Xmx1g"
-Ddruid.indexer.runner.startPort=8088
-Ddruid.indexer.fork.property.druid.processing.numThreads=1
-Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000
```

View File

@ -69,6 +69,20 @@ You should see a bunch of files:
* run_example_client.sh
* LICENSE, config, examples, lib directories
Setting up Zookeeper
--------------------
Before we get started, we need to start Apache Zookeeper.
```bash
curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
tar xzf zookeeper-3.4.5.tar.gz
cd zookeeper-3.4.5
cp conf/zoo_sample.cfg conf/zoo.cfg
./bin/zkServer.sh start
cd ..
```
Running Example Scripts
-----------------------

View File

@ -85,28 +85,23 @@ config/overlord/runtime.properties
The configurations for the overlord node are as follows:
```bash
-server
-Xmx256m
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
druid.host=localhost
druid.port=8087
druid.service=overlord
-Ddruid.host=localhost
-Ddruid.port=8080
-Ddruid.service=overlord
druid.zk.service.host=localhost
-Ddruid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
-Ddruid.db.connector.user=druid
-Ddruid.db.connector.password=diurd
-Ddruid.selectors.indexing.serviceName=overlord
-Ddruid.indexer.queue.startDelay=PT0M
-Ddruid.indexer.runner.javaOpts="-server -Xmx1g"
-Ddruid.indexer.fork.property.druid.processing.numThreads=1
-Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000
druid.selectors.indexing.serviceName=overlord
druid.indexer.queue.startDelay=PT0M
druid.indexer.runner.javaOpts="-server -Xmx256m"
druid.indexer.fork.property.druid.processing.numThreads=1
druid.indexer.fork.property.druid.computation.buffer.size=100000000
```
If you are interested in reading more about these configurations, see [here](Indexing-Service.html).

View File

@ -48,7 +48,7 @@ CREATE database druid;
#### Setting up Zookeeper
```bash
curl http://www.motorlogy.com/apache/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
tar xzf zookeeper-3.4.5.tar.gz
cd zookeeper-3.4.5
cp conf/zoo_sample.cfg conf/zoo.cfg
@ -120,7 +120,7 @@ druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.coordinator.startDelay=PT60s
druid.coordinator.startDelay=PT70s
```
To start the coordinator node:

View File

@ -79,6 +79,7 @@ h2. Experimental
* "About Experimental Features":./About-Experimental-Features.html
* "Geographic Queries":./GeographicQueries.html
* "Select Query":./SelectQuery.html
* "Approximate Histograms and Quantiles":./ApproxHisto.html
h2. Development
* "Versioning":./Versioning.html

View File

@ -1,22 +1,18 @@
-server
-Xmx256m
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
druid.host=localhost
druid.port=8087
druid.service=overlord
-Ddruid.host=localhost
-Ddruid.port=8080
-Ddruid.service=overlord
druid.zk.service.host=localhost
-Ddruid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
-Ddruid.db.connector.user=druid
-Ddruid.db.connector.password=diurd
-Ddruid.selectors.indexing.serviceName=overlord
-Ddruid.indexer.queue.startDelay=PT0M
-Ddruid.indexer.runner.javaOpts="-server -Xmx256m"
-Ddruid.indexer.fork.property.druid.processing.numThreads=1
-Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000
druid.selectors.indexing.serviceName=overlord
druid.indexer.queue.startDelay=PT0M
druid.indexer.runner.javaOpts="-server -Xmx256m"
druid.indexer.runner.startPort=8088
druid.indexer.fork.property.druid.processing.numThreads=1
druid.indexer.fork.property.druid.computation.buffer.size=100000000

View File

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

View File

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

View File

@ -21,6 +21,7 @@ package io.druid.storage.hdfs;
import com.google.common.io.Closeables;
import com.google.inject.Inject;
import com.metamx.common.guava.CloseQuietly;
import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
@ -52,22 +53,17 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller
final FileSystem fs = checkPathAndGetFilesystem(path);
FSDataInputStream in = null;
try {
if (path.getName().endsWith(".zip")) {
in = fs.open(path);
CompressionUtils.unzip(in, dir);
in.close();
if (path.getName().endsWith(".zip")) {
try {
try (FSDataInputStream in = fs.open(path)) {
CompressionUtils.unzip(in, dir);
}
}
else {
throw new SegmentLoadingException("Unknown file type[%s]", path);
catch (IOException e) {
throw new SegmentLoadingException(e, "Some IOException");
}
}
catch (IOException e) {
throw new SegmentLoadingException(e, "Some IOException");
}
finally {
Closeables.closeQuietly(in);
} else {
throw new SegmentLoadingException("Unknown file type[%s]", path);
}
}
@ -85,7 +81,8 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller
}
}
private Path getPath(DataSegment segment) {
private Path getPath(DataSegment segment)
{
return new Path(String.valueOf(segment.getLoadSpec().get("path")));
}

View File

@ -26,6 +26,7 @@ import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.OutputSupplier;
import com.google.inject.Inject;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.logger.Logger;
import io.druid.segment.SegmentUtils;
import io.druid.segment.loading.DataSegmentPusher;
@ -78,17 +79,10 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
fs.mkdirs(outFile.getParent());
log.info("Compressing files from[%s] to [%s]", inDir, outFile);
FSDataOutputStream out = null;
long size;
try {
out = fs.create(outFile);
try (FSDataOutputStream out = fs.create(outFile)) {
size = CompressionUtils.zip(inDir, out);
out.close();
}
finally {
Closeables.closeQuietly(out);
}
return createDescriptorFile(

View File

@ -27,6 +27,8 @@ import io.druid.guice.Binders;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.LazySingleton;
import io.druid.initialization.DruidModule;
import io.druid.storage.hdfs.tasklog.HdfsTaskLogs;
import io.druid.storage.hdfs.tasklog.HdfsTaskLogsConfig;
import org.apache.hadoop.conf.Configuration;
import java.util.List;
@ -68,5 +70,9 @@ public class HdfsStorageDruidModule implements DruidModule
binder.bind(Configuration.class).toInstance(conf);
JsonConfigProvider.bind(binder, "druid.storage", HdfsDataSegmentPusherConfig.class);
Binders.taskLogsBinder(binder).addBinding("hdfs").to(HdfsTaskLogs.class);
JsonConfigProvider.bind(binder, "druid.indexer.logs", HdfsTaskLogsConfig.class);
binder.bind(HdfsTaskLogs.class).in(LazySingleton.class);
}
}

View File

@ -0,0 +1,104 @@
/*
* 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.storage.hdfs.tasklog;
import com.google.common.base.Optional;
import com.google.common.io.ByteStreams;
import com.google.common.io.InputSupplier;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import io.druid.tasklogs.TaskLogs;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
/**
* Indexer hdfs task logs, to support storing hdfs tasks to hdfs
*
* Created by Frank Ren on 6/20/14.
*/
public class HdfsTaskLogs implements TaskLogs
{
private static final Logger log = new Logger(HdfsTaskLogs.class);
private final HdfsTaskLogsConfig config;
@Inject
public HdfsTaskLogs(HdfsTaskLogsConfig config)
{
this.config = config;
}
@Override
public void pushTaskLog(String taskId, File logFile) throws IOException
{
final Path path = getTaskLogFileFromId(taskId);
log.info("writing task log to: %s", path);
Configuration conf = new Configuration();
final FileSystem fs = FileSystem.get(conf);
FileUtil.copy(logFile, fs, path, false, conf);
log.info("wrote task log to: %s", path);
}
@Override
public Optional<InputSupplier<InputStream>> streamTaskLog(final String taskId, final long offset) throws IOException
{
final Path path = getTaskLogFileFromId(taskId);
final FileSystem fs = FileSystem.get(new Configuration());
if (fs.exists(path)) {
return Optional.<InputSupplier<InputStream>>of(
new InputSupplier<InputStream>() {
@Override
public InputStream getInput() throws IOException
{
log.info("reading task log from: %s", path);
final InputStream inputStream = fs.open(path);
ByteStreams.skipFully(inputStream, offset);
log.info("read task log from: %s", path);
return inputStream;
}
}
);
} else {
return Optional.absent();
}
}
/**
* Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in
* path names. So we format paths differently for HDFS.
*/
private Path getTaskLogFileFromId(String taskId)
{
return new Path(mergePaths(config.getDirectory(), taskId.replaceAll(":", "_")));
}
// some hadoop version Path.mergePaths does not exist
private static String mergePaths(String path1, String path2)
{
return path1 + (path1.endsWith(Path.SEPARATOR) ? "" : Path.SEPARATOR) + path2;
}
}

View File

@ -16,38 +16,26 @@
* 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.storage.hdfs.tasklog;
package io.druid.segment.data;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Iterators;
import java.util.Iterator;
import javax.validation.constraints.NotNull;
/**
*/
public class SingleIndexedInts implements IndexedInts
* Indexer hdfs task logs configuration
*
* Created by Frank Ren on 6/20/14.
*/
public class HdfsTaskLogsConfig
{
private final int value;
@JsonProperty
@NotNull
private String directory;
public SingleIndexedInts(int value) {
this.value = value;
}
@Override
public int size()
{
return 1;
}
@Override
public int get(int index)
{
return value;
}
@Override
public Iterator<Integer> iterator()
{
return Iterators.singletonIterator(value);
}
public String getDirectory()
{
return directory;
}
}

71
histogram/pom.xml Normal file
View File

@ -0,0 +1,71 @@
<?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-histogram</artifactId>
<name>druid-histogram</name>
<description>druid-histogram</description>
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
</dependency>
<!-- Tests -->
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<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,103 @@
/*
* 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.aggregation.histogram;
import com.google.common.primitives.Longs;
import io.druid.query.aggregation.Aggregator;
import io.druid.segment.FloatColumnSelector;
import java.util.Comparator;
public class ApproximateHistogramAggregator implements Aggregator
{
public static final Comparator COMPARATOR = new Comparator()
{
@Override
public int compare(Object o, Object o1)
{
return Longs.compare(((ApproximateHistogram) o).count(), ((ApproximateHistogram) o1).count());
}
};
static Object combineHistograms(Object lhs, Object rhs)
{
return ((ApproximateHistogram) lhs).foldFast((ApproximateHistogram) rhs);
}
private final String name;
private final FloatColumnSelector selector;
private final int resolution;
private final float lowerLimit;
private final float upperLimit;
private ApproximateHistogram histogram;
public ApproximateHistogramAggregator(
String name,
FloatColumnSelector selector,
int resolution,
float lowerLimit,
float upperLimit
)
{
this.name = name;
this.selector = selector;
this.resolution = resolution;
this.lowerLimit = lowerLimit;
this.upperLimit = upperLimit;
this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit);
}
@Override
public void aggregate()
{
histogram.offer(selector.get());
}
@Override
public void reset()
{
this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit);
}
@Override
public Object get()
{
return histogram;
}
@Override
public float getFloat()
{
throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getFloat()");
}
@Override
public String getName()
{
return name;
}
@Override
public void close()
{
// no resources to cleanup
}
}

View File

@ -0,0 +1,253 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.BufferAggregator;
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;
@JsonTypeName("approxHistogram")
public class ApproximateHistogramAggregatorFactory implements AggregatorFactory
{
private static final byte CACHE_TYPE_ID = 0x8;
protected final String name;
protected final String fieldName;
protected final int resolution;
protected final int numBuckets;
protected final float lowerLimit;
protected final float upperLimit;
@JsonCreator
public ApproximateHistogramAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("resolution") Integer resolution,
@JsonProperty("numBuckets") Integer numBuckets,
@JsonProperty("lowerLimit") Float lowerLimit,
@JsonProperty("upperLimit") Float upperLimit
)
{
this.name = name;
this.fieldName = fieldName.toLowerCase();
this.resolution = resolution == null ? ApproximateHistogram.DEFAULT_HISTOGRAM_SIZE : resolution;
this.numBuckets = numBuckets == null ? ApproximateHistogram.DEFAULT_BUCKET_SIZE : numBuckets;
this.lowerLimit = lowerLimit == null ? Float.NEGATIVE_INFINITY : lowerLimit;
this.upperLimit = upperLimit == null ? Float.POSITIVE_INFINITY : upperLimit;
Preconditions.checkArgument(this.resolution > 0, "resolution must be greater than 1");
Preconditions.checkArgument(this.numBuckets > 0, "numBuckets must be greater than 1");
Preconditions.checkArgument(this.upperLimit > this.lowerLimit, "upperLimit must be greater than lowerLimit");
}
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
return new ApproximateHistogramAggregator(
name,
metricFactory.makeFloatColumnSelector(fieldName),
resolution,
lowerLimit,
upperLimit
);
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
return new ApproximateHistogramBufferAggregator(
metricFactory.makeFloatColumnSelector(fieldName),
resolution,
lowerLimit,
upperLimit
);
}
@Override
public Comparator getComparator()
{
return ApproximateHistogramAggregator.COMPARATOR;
}
@Override
public Object combine(Object lhs, Object rhs)
{
return ApproximateHistogramAggregator.combineHistograms(lhs, rhs);
}
@Override
public AggregatorFactory getCombiningFactory()
{
return new ApproximateHistogramAggregatorFactory(name, name, resolution, numBuckets, lowerLimit, upperLimit);
}
@Override
public List<AggregatorFactory> getRequiredColumns()
{
return Arrays.<AggregatorFactory>asList(
new ApproximateHistogramAggregatorFactory(
fieldName,
fieldName,
resolution,
numBuckets,
lowerLimit,
upperLimit
)
);
}
@Override
public Object deserialize(Object object)
{
if (object instanceof byte[]) {
final ApproximateHistogram ah = ApproximateHistogram.fromBytes((byte[]) object);
ah.setLowerLimit(lowerLimit);
ah.setUpperLimit(upperLimit);
return ah;
} else if (object instanceof ByteBuffer) {
final ApproximateHistogram ah = ApproximateHistogram.fromBytes((ByteBuffer) object);
ah.setLowerLimit(lowerLimit);
ah.setUpperLimit(upperLimit);
return ah;
} else if (object instanceof String) {
byte[] bytes = Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8));
final ApproximateHistogram ah = ApproximateHistogram.fromBytes(bytes);
ah.setLowerLimit(lowerLimit);
ah.setUpperLimit(upperLimit);
return ah;
} else {
return object;
}
}
@Override
public Object finalizeComputation(Object object)
{
return ((ApproximateHistogram) object).toHistogram(numBuckets);
}
@JsonProperty
@Override
public String getName()
{
return name;
}
@JsonProperty
public String getFieldName()
{
return fieldName;
}
@JsonProperty
public int getResolution()
{
return resolution;
}
@JsonProperty
public float getLowerLimit()
{
return lowerLimit;
}
@JsonProperty
public float getUpperLimit()
{
return upperLimit;
}
@JsonProperty
public int getNumBuckets()
{
return numBuckets;
}
@Override
public List<String> requiredFields()
{
return Arrays.asList(fieldName);
}
@Override
public byte[] getCacheKey()
{
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8);
return ByteBuffer.allocate(1 + fieldNameBytes.length + Ints.BYTES * 2 + Floats.BYTES * 2)
.put(CACHE_TYPE_ID)
.put(fieldNameBytes)
.putInt(resolution)
.putInt(numBuckets)
.putFloat(lowerLimit)
.putFloat(upperLimit).array();
}
@Override
public String getTypeName()
{
return "approximateHistogram";
}
@Override
public int getMaxIntermediateSize()
{
return new ApproximateHistogram(resolution).getMaxStorageSize();
}
@Override
public Object getAggregatorStartValue()
{
return new ApproximateHistogram(resolution);
}
@Override
public String toString()
{
return "ApproximateHistogramAggregatorFactory{" +
"name='" + name + '\'' +
", fieldName='" + fieldName + '\'' +
", resolution=" + resolution +
", numBuckets=" + numBuckets +
", lowerLimit=" + lowerLimit +
", upperLimit=" + upperLimit +
'}';
}
}

View File

@ -0,0 +1,95 @@
/*
* 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.aggregation.histogram;
import io.druid.query.aggregation.BufferAggregator;
import io.druid.segment.FloatColumnSelector;
import java.nio.ByteBuffer;
public class ApproximateHistogramBufferAggregator implements BufferAggregator
{
private final FloatColumnSelector selector;
private final int resolution;
private final float lowerLimit;
private final float upperLimit;
public ApproximateHistogramBufferAggregator(FloatColumnSelector selector, int resolution, float lowerLimit, float upperLimit)
{
this.selector = selector;
this.resolution = resolution;
this.lowerLimit = lowerLimit;
this.upperLimit = upperLimit;
}
@Override
public void init(ByteBuffer buf, int position)
{
ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
mutationBuffer.putInt(resolution);
mutationBuffer.putInt(0); //initial binCount
for (int i = 0; i < resolution; ++i) {
mutationBuffer.putFloat(0f);
}
for (int i = 0; i < resolution; ++i) {
mutationBuffer.putLong(0L);
}
// min
mutationBuffer.putFloat(Float.POSITIVE_INFINITY);
// max
mutationBuffer.putFloat(Float.NEGATIVE_INFINITY);
}
@Override
public void aggregate(ByteBuffer buf, int position)
{
ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
ApproximateHistogram h0 = ApproximateHistogram.fromBytesDense(mutationBuffer);
h0.offer(selector.get());
mutationBuffer.position(position);
h0.toBytesDense(mutationBuffer);
}
@Override
public Object get(ByteBuffer buf, int position)
{
ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
return ApproximateHistogram.fromBytes(mutationBuffer);
}
@Override
public float getFloat(ByteBuffer buf, int position)
{
throw new UnsupportedOperationException("ApproximateHistogramBufferAggregator does not support getFloat()");
}
@Override
public void close()
{
// no resources to cleanup
}
}

View File

@ -0,0 +1,60 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import io.druid.initialization.DruidModule;
import io.druid.segment.serde.ComplexMetrics;
import java.util.List;
/**
*/
public class ApproximateHistogramDruidModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(
new SimpleModule().registerSubtypes(
ApproximateHistogramFoldingAggregatorFactory.class,
ApproximateHistogramAggregatorFactory.class,
EqualBucketsPostAggregator.class,
CustomBucketsPostAggregator.class,
BucketsPostAggregator.class,
QuantilesPostAggregator.class,
QuantilePostAggregator.class,
MinPostAggregator.class,
MaxPostAggregator.class
)
);
}
@Override
public void configure(Binder binder)
{
if (ComplexMetrics.getSerdeForType("approximateHistogram") == null) {
ComplexMetrics.registerSerde("approximateHistogram", new ApproximateHistogramFoldingSerde());
}
}
}

View File

@ -0,0 +1,101 @@
/*
* 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.aggregation.histogram;
import io.druid.query.aggregation.Aggregator;
import io.druid.segment.ObjectColumnSelector;
public class ApproximateHistogramFoldingAggregator implements Aggregator
{
private final String name;
private final ObjectColumnSelector<ApproximateHistogram> selector;
private final int resolution;
private final float lowerLimit;
private final float upperLimit;
private ApproximateHistogram histogram;
private float[] tmpBufferP;
private long[] tmpBufferB;
public ApproximateHistogramFoldingAggregator(
String name,
ObjectColumnSelector<ApproximateHistogram> selector,
int resolution,
float lowerLimit,
float upperLimit
)
{
this.name = name;
this.selector = selector;
this.resolution = resolution;
this.lowerLimit = lowerLimit;
this.upperLimit = upperLimit;
this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit);
tmpBufferP = new float[resolution];
tmpBufferB = new long[resolution];
}
@Override
public void aggregate()
{
ApproximateHistogram h = selector.get();
if (h == null) {
return;
}
if (h.binCount() + histogram.binCount() <= tmpBufferB.length) {
histogram.foldFast(h, tmpBufferP, tmpBufferB);
} else {
histogram.foldFast(h);
}
}
@Override
public void reset()
{
this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit);
}
@Override
public Object get()
{
return histogram;
}
@Override
public float getFloat()
{
throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getFloat()");
}
@Override
public String getName()
{
return name;
}
@Override
public void close()
{
// no resources to cleanup
}
}

View File

@ -0,0 +1,164 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Charsets;
import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints;
import com.metamx.common.IAE;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.BufferAggregator;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.ObjectColumnSelector;
import java.nio.ByteBuffer;
@JsonTypeName("approxHistogramFold")
public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHistogramAggregatorFactory
{
private static final byte CACHE_TYPE_ID = 0x9;
@JsonCreator
public ApproximateHistogramFoldingAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("resolution") Integer resolution,
@JsonProperty("numBuckets") Integer numBuckets,
@JsonProperty("lowerLimit") Float lowerLimit,
@JsonProperty("upperLimit") Float upperLimit
)
{
super(name, fieldName, resolution, numBuckets, lowerLimit, upperLimit);
}
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName);
if (selector == null) {
// gracefully handle undefined metrics
selector = new ObjectColumnSelector<ApproximateHistogram>()
{
@Override
public Class<ApproximateHistogram> classOfObject()
{
return ApproximateHistogram.class;
}
@Override
public ApproximateHistogram get()
{
return new ApproximateHistogram(0);
}
};
}
if (ApproximateHistogram.class.isAssignableFrom(selector.classOfObject())) {
return new ApproximateHistogramFoldingAggregator(
name,
selector,
resolution,
lowerLimit,
upperLimit
);
}
throw new IAE(
"Incompatible type for metric[%s], expected a ApproximateHistogram, got a %s",
fieldName,
selector.classOfObject()
);
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName);
if (selector == null) {
// gracefully handle undefined metrics
selector = new ObjectColumnSelector<ApproximateHistogram>()
{
@Override
public Class<ApproximateHistogram> classOfObject()
{
return ApproximateHistogram.class;
}
@Override
public ApproximateHistogram get()
{
return new ApproximateHistogram(0);
}
};
}
if (ApproximateHistogram.class.isAssignableFrom(selector.classOfObject())) {
return new ApproximateHistogramFoldingBufferAggregator(selector, resolution, lowerLimit, upperLimit);
}
throw new IAE(
"Incompatible type for metric[%s], expected a ApproximateHistogram, got a %s",
fieldName,
selector.classOfObject()
);
}
@Override
public AggregatorFactory getCombiningFactory()
{
return new ApproximateHistogramFoldingAggregatorFactory(name, name, resolution, numBuckets, lowerLimit, upperLimit);
}
@Override
public byte[] getCacheKey()
{
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8);
return ByteBuffer.allocate(1 + fieldNameBytes.length + Ints.BYTES * 2 + Floats.BYTES * 2)
.put(CACHE_TYPE_ID)
.put(fieldNameBytes)
.putInt(resolution)
.putInt(numBuckets)
.putFloat(lowerLimit)
.putFloat(upperLimit)
.array();
}
@Override
public String toString()
{
return "ApproximateHistogramFoldingAggregatorFactory{" +
"name='" + name + '\'' +
", fieldName='" + fieldName + '\'' +
", resolution=" + resolution +
", numBuckets=" + numBuckets +
", lowerLimit=" + lowerLimit +
", upperLimit=" + upperLimit +
'}';
}
}

View File

@ -0,0 +1,99 @@
/*
* 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.aggregation.histogram;
import io.druid.query.aggregation.BufferAggregator;
import io.druid.segment.ObjectColumnSelector;
import java.nio.ByteBuffer;
public class ApproximateHistogramFoldingBufferAggregator implements BufferAggregator
{
private final ObjectColumnSelector<ApproximateHistogram> selector;
private final int resolution;
private final float upperLimit;
private final float lowerLimit;
private float[] tmpBufferP;
private long[] tmpBufferB;
public ApproximateHistogramFoldingBufferAggregator(
ObjectColumnSelector<ApproximateHistogram> selector,
int resolution,
float lowerLimit,
float upperLimit
)
{
this.selector = selector;
this.resolution = resolution;
this.lowerLimit = lowerLimit;
this.upperLimit = upperLimit;
tmpBufferP = new float[resolution];
tmpBufferB = new long[resolution];
}
@Override
public void init(ByteBuffer buf, int position)
{
ApproximateHistogram h = new ApproximateHistogram(resolution, lowerLimit, upperLimit);
ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
// use dense storage for aggregation
h.toBytesDense(mutationBuffer);
}
@Override
public void aggregate(ByteBuffer buf, int position)
{
ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
ApproximateHistogram h0 = ApproximateHistogram.fromBytesDense(mutationBuffer);
h0.setLowerLimit(lowerLimit);
h0.setUpperLimit(upperLimit);
ApproximateHistogram hNext = selector.get();
h0.foldFast(hNext, tmpBufferP, tmpBufferB);
mutationBuffer.position(position);
h0.toBytesDense(mutationBuffer);
}
@Override
public Object get(ByteBuffer buf, int position)
{
ByteBuffer mutationBuffer = buf.asReadOnlyBuffer();
mutationBuffer.position(position);
return ApproximateHistogram.fromBytesDense(mutationBuffer);
}
@Override
public float getFloat(ByteBuffer buf, int position)
{
throw new UnsupportedOperationException("ApproximateHistogramFoldingBufferAggregator does not support getFloat()");
}
@Override
public void close()
{
// no resources to cleanup
}
}

View File

@ -0,0 +1,140 @@
/*
* 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.aggregation.histogram;
import com.google.common.collect.Ordering;
import io.druid.data.input.InputRow;
import io.druid.segment.column.ColumnBuilder;
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.Iterator;
import java.util.List;
public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde
{
private static Ordering<ApproximateHistogram> comparator = new Ordering<ApproximateHistogram>()
{
@Override
public int compare(
ApproximateHistogram arg1, ApproximateHistogram arg2
)
{
return ApproximateHistogramAggregator.COMPARATOR.compare(arg1, arg2);
}
}.nullsFirst();
@Override
public String getTypeName()
{
return "approximateHistogram";
}
@Override
public ComplexMetricExtractor getExtractor()
{
return new ComplexMetricExtractor()
{
@Override
public Class<ApproximateHistogram> extractedClass()
{
return ApproximateHistogram.class;
}
@Override
public ApproximateHistogram extractValue(InputRow inputRow, String metricName)
{
Object rawValue = inputRow.getRaw(metricName);
if (rawValue instanceof ApproximateHistogram) {
return (ApproximateHistogram) rawValue;
} else {
List<String> dimValues = inputRow.getDimension(metricName);
if (dimValues != null && dimValues.size() > 0) {
Iterator<String> values = dimValues.iterator();
ApproximateHistogram h = new ApproximateHistogram();
while (values.hasNext()) {
float value = Float.parseFloat(values.next());
h.offer(value);
}
return h;
} else {
return new ApproximateHistogram(0);
}
}
}
};
}
@Override
public ColumnPartSerde deserializeColumn(
ByteBuffer byteBuffer, ColumnBuilder columnBuilder
)
{
final GenericIndexed column = GenericIndexed.read(byteBuffer, getObjectStrategy());
columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column));
return new ComplexColumnPartSerde(column, getTypeName());
}
public ObjectStrategy getObjectStrategy()
{
return new ObjectStrategy<ApproximateHistogram>()
{
@Override
public Class<? extends ApproximateHistogram> getClazz()
{
return ApproximateHistogram.class;
}
@Override
public ApproximateHistogram fromByteBuffer(ByteBuffer buffer, int numBytes)
{
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
return ApproximateHistogram.fromBytes(readOnlyBuffer);
}
@Override
public byte[] toBytes(ApproximateHistogram h)
{
if (h == null) {
return new byte[]{};
}
return h.toBytes();
}
@Override
public int compare(ApproximateHistogram o1, ApproximateHistogram o2)
{
return comparator.compare(o1, o2);
}
};
}
}

View File

@ -0,0 +1,68 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.query.aggregation.PostAggregator;
import java.util.Comparator;
import java.util.Map;
public abstract class ApproximateHistogramPostAggregator implements PostAggregator
{
private static final Comparator COMPARATOR = ApproximateHistogramAggregator.COMPARATOR;
private final String name;
private final String fieldName;
public ApproximateHistogramPostAggregator(
String name,
String fieldName
)
{
this.name = name;
this.fieldName = fieldName;
}
@Override
public Comparator getComparator()
{
return COMPARATOR;
}
@Override
public abstract Object compute(Map<String, Object> values);
@Override
@JsonProperty
public String getName()
{
return name;
}
@JsonProperty
public String getFieldName()
{
return fieldName;
}
@Override
public abstract String toString();
}

View File

@ -0,0 +1,58 @@
/*
* 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.aggregation.histogram;
public class ArrayUtils
{
public static int hashCode(long[] a, int fromIndex, int toIndex)
{
int hashCode = 1;
int i = fromIndex;
while (i < toIndex) {
long v = a[i];
hashCode = 31 * hashCode + (int) (v ^ (v >>> 32));
++i;
}
return hashCode;
}
public static int hashCode(float[] a, int fromIndex, int toIndex)
{
int hashCode = 1;
int i = fromIndex;
while (i < toIndex) {
hashCode = 31 * hashCode + Float.floatToIntBits(a[i]);
++i;
}
return hashCode;
}
public static int hashCode(double[] a, int fromIndex, int toIndex)
{
int hashCode = 1;
int i = fromIndex;
while (i < toIndex) {
long v = Double.doubleToLongBits(a[i]);
hashCode = 31 * hashCode + (int) (v ^ (v >>> 32));
++i;
}
return hashCode;
}
}

View File

@ -0,0 +1,91 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Sets;
import com.metamx.common.IAE;
import java.util.Map;
import java.util.Set;
@JsonTypeName("buckets")
public class BucketsPostAggregator extends ApproximateHistogramPostAggregator
{
private final float bucketSize;
private final float offset;
private String fieldName;
@JsonCreator
public BucketsPostAggregator(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("bucketSize") float bucketSize,
@JsonProperty("offset") float offset
)
{
super(name, fieldName);
this.bucketSize = bucketSize;
if (this.bucketSize <= 0) {
throw new IAE("Illegal bucketSize [%s], must be > 0", this.bucketSize);
}
this.offset = offset;
this.fieldName = fieldName;
}
@Override
public Set<String> getDependentFields()
{
return Sets.newHashSet(fieldName);
}
@Override
public Object compute(Map<String, Object> values)
{
ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName());
return ah.toHistogram(bucketSize, offset);
}
@JsonProperty
public float getBucketSize()
{
return bucketSize;
}
@JsonProperty
public float getOffset()
{
return bucketSize;
}
@Override
public String toString()
{
return "BucketsPostAggregator{" +
"name='" + this.getName() + '\'' +
", fieldName='" + this.getFieldName() + '\'' +
", bucketSize=" + this.getBucketSize() +
", offset=" + this.getOffset() +
'}';
}
}

View File

@ -0,0 +1,68 @@
/*
* 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.aggregation.histogram;
import java.nio.DoubleBuffer;
import java.nio.FloatBuffer;
public class BufferUtils
{
public static int binarySearch(DoubleBuffer buf, int minIndex, int maxIndex, double value)
{
while (minIndex < maxIndex) {
int currIndex = (minIndex + maxIndex - 1) >>> 1;
double currValue = buf.get(currIndex);
int comparison = Double.compare(currValue, value);
if (comparison == 0) {
return currIndex;
}
if (comparison < 0) {
minIndex = currIndex + 1;
} else {
maxIndex = currIndex;
}
}
return -(minIndex + 1);
}
public static int binarySearch(FloatBuffer buf, int minIndex, int maxIndex, float value)
{
while (minIndex < maxIndex) {
int currIndex = (minIndex + maxIndex - 1) >>> 1;
float currValue = buf.get(currIndex);
int comparison = Float.compare(currValue, value);
if (comparison == 0) {
return currIndex;
}
if (comparison < 0) {
minIndex = currIndex + 1;
} else {
maxIndex = currIndex;
}
}
return -(minIndex + 1);
}
}

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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Sets;
import java.util.Arrays;
import java.util.Map;
import java.util.Set;
@JsonTypeName("customBuckets")
public class CustomBucketsPostAggregator extends ApproximateHistogramPostAggregator
{
private final float[] breaks;
private String fieldName;
@JsonCreator
public CustomBucketsPostAggregator(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("breaks") float[] breaks
)
{
super(name, fieldName);
this.breaks = breaks;
this.fieldName = fieldName;
}
@Override
public Set<String> getDependentFields()
{
return Sets.newHashSet(fieldName);
}
@Override
public Object compute(Map<String, Object> values)
{
ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName());
return ah.toHistogram(breaks);
}
@JsonProperty
public float[] getBreaks()
{
return breaks;
}
@Override
public String toString()
{
return "CustomBucketsPostAggregator{" +
"name='" + this.getName() + '\'' +
", fieldName='" + this.getFieldName() + '\'' +
", breaks=" + Arrays.toString(this.getBreaks()) +
'}';
}
}

View File

@ -0,0 +1,80 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Sets;
import com.metamx.common.IAE;
import java.util.Map;
import java.util.Set;
@JsonTypeName("equalBuckets")
public class EqualBucketsPostAggregator extends ApproximateHistogramPostAggregator
{
private final int numBuckets;
private String fieldName;
@JsonCreator
public EqualBucketsPostAggregator(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("numBuckets") int numBuckets
)
{
super(name, fieldName);
this.numBuckets = numBuckets;
if (this.numBuckets <= 1) {
throw new IAE("Illegal number of buckets[%s], must be > 1", this.numBuckets);
}
this.fieldName = fieldName;
}
@Override
public Set<String> getDependentFields()
{
return Sets.newHashSet(fieldName);
}
@Override
public Object compute(Map<String, Object> values)
{
ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName());
return ah.toHistogram(numBuckets);
}
@JsonProperty
public int getNumBuckets()
{
return numBuckets;
}
@Override
public String toString()
{
return "EqualBucketsPostAggregator{" +
"name='" + this.getName() + '\'' +
", fieldName='" + this.getFieldName() + '\'' +
", numBuckets=" + this.getNumBuckets() +
'}';
}
}

View File

@ -0,0 +1,88 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Arrays;
public class Histogram
{
double[] breaks;
double[] counts;
public Histogram(float[] breaks, double[] counts)
{
double[] retVal = new double[breaks.length];
for (int i = 0; i < breaks.length; ++i) {
retVal[i] = (double) breaks[i];
}
this.breaks = retVal;
this.counts = counts;
}
@JsonProperty
public double[] getBreaks()
{
return breaks;
}
@JsonProperty
public double[] getCounts()
{
return counts;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Histogram that = (Histogram) o;
if (!Arrays.equals(this.getBreaks(), that.getBreaks())) {
return false;
}
if (!Arrays.equals(this.getCounts(), that.getCounts())) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = (this.getBreaks() != null ? ArrayUtils.hashCode(this.getBreaks(), 0, this.getBreaks().length) : 0);
result = 31 * result + (this.getCounts() != null ? ArrayUtils.hashCode(
this.getCounts(),
0,
this.getCounts().length
) : 0);
return result;
}
}

View File

@ -0,0 +1,81 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Sets;
import java.util.Comparator;
import java.util.Map;
import java.util.Set;
@JsonTypeName("max")
public class MaxPostAggregator extends ApproximateHistogramPostAggregator
{
static final Comparator COMPARATOR = new Comparator()
{
@Override
public int compare(Object o, Object o1)
{
return Double.compare(((Number) o).doubleValue(), ((Number) o1).doubleValue());
}
};
private String fieldName;
@JsonCreator
public MaxPostAggregator(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName
)
{
super(name, fieldName);
this.fieldName = fieldName;
}
@Override
public Comparator getComparator()
{
return COMPARATOR;
}
@Override
public Set<String> getDependentFields()
{
return Sets.newHashSet(fieldName);
}
@Override
public Object compute(Map<String, Object> values)
{
final ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName());
return ah.getMax();
}
@Override
public String toString()
{
return "QuantilePostAggregator{" +
"fieldName='" + fieldName + '\'' +
'}';
}
}

View File

@ -0,0 +1,81 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Sets;
import java.util.Comparator;
import java.util.Map;
import java.util.Set;
@JsonTypeName("min")
public class MinPostAggregator extends ApproximateHistogramPostAggregator
{
static final Comparator COMPARATOR = new Comparator()
{
@Override
public int compare(Object o, Object o1)
{
return Double.compare(((Number) o).doubleValue(), ((Number) o1).doubleValue());
}
};
private String fieldName;
@JsonCreator
public MinPostAggregator(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName
)
{
super(name, fieldName);
this.fieldName = fieldName;
}
@Override
public Comparator getComparator()
{
return COMPARATOR;
}
@Override
public Set<String> getDependentFields()
{
return Sets.newHashSet(fieldName);
}
@Override
public Object compute(Map<String, Object> values)
{
final ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName());
return ah.getMin();
}
@Override
public String toString()
{
return "QuantilePostAggregator{" +
"fieldName='" + fieldName + '\'' +
'}';
}
}

View File

@ -0,0 +1,96 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Sets;
import com.metamx.common.IAE;
import java.util.Comparator;
import java.util.Map;
import java.util.Set;
@JsonTypeName("quantile")
public class QuantilePostAggregator extends ApproximateHistogramPostAggregator
{
static final Comparator COMPARATOR = new Comparator()
{
@Override
public int compare(Object o, Object o1)
{
return Double.compare(((Number) o).doubleValue(), ((Number) o1).doubleValue());
}
};
private final float probability;
private String fieldName;
@JsonCreator
public QuantilePostAggregator(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("probability") float probability
)
{
super(name, fieldName);
this.probability = probability;
this.fieldName = fieldName;
if (probability < 0 | probability > 1) {
throw new IAE("Illegal probability[%s], must be strictly between 0 and 1", probability);
}
}
@Override
public Comparator getComparator()
{
return COMPARATOR;
}
@Override
public Set<String> getDependentFields()
{
return Sets.newHashSet(fieldName);
}
@Override
public Object compute(Map<String, Object> values)
{
final ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName());
return ah.getQuantiles(new float[]{this.getProbability()})[0];
}
@JsonProperty
public float getProbability()
{
return probability;
}
@Override
public String toString()
{
return "QuantilePostAggregator{" +
"probability=" + probability +
", fieldName='" + fieldName + '\'' +
'}';
}
}

View File

@ -0,0 +1,111 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import java.util.Arrays;
@JsonTypeName("quantiles")
public class Quantiles
{
float[] probabilities;
float[] quantiles;
float min;
float max;
@JsonCreator
public Quantiles(
@JsonProperty("probabilities") float[] probabilities,
@JsonProperty("quantiles") float[] quantiles,
@JsonProperty("min") float min,
@JsonProperty("max") float max
)
{
this.probabilities = probabilities;
this.quantiles = quantiles;
this.min = min;
this.max = max;
}
@JsonProperty
public float[] getProbabilities()
{
return probabilities;
}
@JsonProperty
public float[] getQuantiles()
{
return quantiles;
}
@JsonProperty
public float getMin()
{
return min;
}
@JsonProperty
public float getMax()
{
return max;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Quantiles quantiles1 = (Quantiles) o;
if (Float.compare(quantiles1.max, max) != 0) {
return false;
}
if (Float.compare(quantiles1.min, min) != 0) {
return false;
}
if (!Arrays.equals(probabilities, quantiles1.probabilities)) {
return false;
}
if (!Arrays.equals(quantiles, quantiles1.quantiles)) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = probabilities != null ? Arrays.hashCode(probabilities) : 0;
result = 31 * result + (quantiles != null ? Arrays.hashCode(quantiles) : 0);
result = 31 * result + (min != +0.0f ? Float.floatToIntBits(min) : 0);
result = 31 * result + (max != +0.0f ? Float.floatToIntBits(max) : 0);
return result;
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Sets;
import com.metamx.common.IAE;
import java.util.Arrays;
import java.util.Comparator;
import java.util.Map;
import java.util.Set;
@JsonTypeName("quantiles")
public class QuantilesPostAggregator extends ApproximateHistogramPostAggregator
{
private final float[] probabilities;
private String fieldName;
@JsonCreator
public QuantilesPostAggregator(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("probabilities") float[] probabilities
)
{
super(name, fieldName);
this.probabilities = probabilities;
this.fieldName = fieldName;
for (float p : probabilities) {
if (p < 0 | p > 1) {
throw new IAE("Illegal probability[%s], must be strictly between 0 and 1", p);
}
}
}
@Override
public Comparator getComparator()
{
throw new UnsupportedOperationException();
}
@Override
public Set<String> getDependentFields()
{
return Sets.newHashSet(fieldName);
}
@Override
public Object compute(Map<String, Object> values)
{
final ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName());
return new Quantiles(this.getProbabilities(), ah.getQuantiles(this.getProbabilities()), ah.getMin(), ah.getMax());
}
@JsonProperty
public float[] getProbabilities()
{
return probabilities;
}
@Override
public String toString()
{
return "EqualBucketsPostAggregator{" +
"name='" + this.getName() + '\'' +
", fieldName='" + this.getFieldName() + '\'' +
", probabilities=" + Arrays.toString(this.getProbabilities()) +
'}';
}
}

View File

@ -0,0 +1 @@
io.druid.query.aggregation.histogram.ApproximateHistogramDruidModule

View File

@ -0,0 +1,76 @@
/*
* 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.aggregation.histogram;
import io.druid.query.aggregation.BufferAggregator;
import io.druid.query.aggregation.TestFloatColumnSelector;
import org.junit.Assert;
import org.junit.Test;
import java.nio.ByteBuffer;
public class ApproximateHistogramAggregatorTest
{
private void aggregateBuffer(TestFloatColumnSelector selector, BufferAggregator agg, ByteBuffer buf, int position)
{
agg.aggregate(buf, position);
selector.increment();
}
@Test
public void testBufferAggregate() throws Exception
{
final float[] values = {23, 19, 10, 16, 36, 2, 9, 32, 30, 45};
final int resolution = 5;
final int numBuckets = 5;
final TestFloatColumnSelector selector = new TestFloatColumnSelector(values);
ApproximateHistogramAggregatorFactory factory = new ApproximateHistogramAggregatorFactory(
"billy", "billy", resolution, numBuckets, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY
);
ApproximateHistogramBufferAggregator agg = new ApproximateHistogramBufferAggregator(selector, resolution, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY);
ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize());
int position = 0;
agg.init(buf, position);
for (int i = 0; i < values.length; i++) {
aggregateBuffer(selector, agg, buf, position);
}
ApproximateHistogram h = ((ApproximateHistogram) agg.get(buf, position));
Assert.assertArrayEquals(
"final bin positions don't match expected positions",
new float[]{2, 9.5f, 19.33f, 32.67f, 45f}, h.positions, 0.01f
);
Assert.assertArrayEquals(
"final bin counts don't match expected counts",
new long[]{1, 2, 3, 3, 1}, h.bins()
);
Assert.assertEquals("getMin value doesn't match expected getMin", 2, h.min(), 0);
Assert.assertEquals("getMax value doesn't match expected getMax", 45, h.max(), 0);
Assert.assertEquals("bin count doesn't match expected bin count", 5, h.binCount());
}
}

View File

@ -0,0 +1,191 @@
/*
* 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.aggregation.histogram;
import com.google.common.primitives.Floats;
import io.druid.query.aggregation.Histogram;
import java.util.Arrays;
import java.util.Random;
public class ApproximateHistogramErrorBenchmark
{
private boolean debug = true;
private int numBuckets = 20;
private int numBreaks = numBuckets + 1;
private int numPerHist = 50;
private int numHists = 10;
private int resolution = 50;
private int combinedResolution = 100;
private Random rand = new Random(2);
public ApproximateHistogramErrorBenchmark setDebug(boolean debug)
{
this.debug = debug;
return this;
}
public ApproximateHistogramErrorBenchmark setNumBuckets(int numBuckets)
{
this.numBuckets = numBuckets;
return this;
}
public ApproximateHistogramErrorBenchmark setNumBreaks(int numBreaks)
{
this.numBreaks = numBreaks;
return this;
}
public ApproximateHistogramErrorBenchmark setNumPerHist(int numPerHist)
{
this.numPerHist = numPerHist;
return this;
}
public ApproximateHistogramErrorBenchmark setNumHists(int numHists)
{
this.numHists = numHists;
return this;
}
public ApproximateHistogramErrorBenchmark setResolution(int resolution)
{
this.resolution = resolution;
return this;
}
public ApproximateHistogramErrorBenchmark setCombinedResolution(int combinedResolution)
{
this.combinedResolution = combinedResolution;
return this;
}
public static void main(String[] args)
{
ApproximateHistogramErrorBenchmark approxHist = new ApproximateHistogramErrorBenchmark();
System.out.println(
Arrays.toString(
approxHist.setDebug(true)
.setNumPerHist(50)
.setNumHists(10000)
.setResolution(50)
.setCombinedResolution(100)
.getErrors()
)
);
ApproximateHistogramErrorBenchmark approxHist2 = new ApproximateHistogramErrorBenchmark();
int[] numHistsArray = new int[]{10, 100, 1000, 10000, 100000};
float[] errs1 = new float[numHistsArray.length];
float[] errs2 = new float[numHistsArray.length];
for (int i = 0; i < numHistsArray.length; ++i) {
float[] tmp = approxHist2.setDebug(false).setNumHists(numHistsArray[i]).setCombinedResolution(100).getErrors();
errs1[i] = tmp[0];
errs2[i] = tmp[1];
}
System.out
.format("Number of histograms for folding : %s \n", Arrays.toString(numHistsArray));
System.out.format("Errors for approximate histogram : %s \n", Arrays.toString(errs1));
System.out.format("Errors for approximate histogram, ruleFold : %s \n", Arrays.toString(errs2));
}
private float[] getErrors()
{
final int numValues = numHists * numPerHist;
final float[] values = new float[numValues];
for (int i = 0; i < numValues; ++i) {
values[i] = (float) rand.nextGaussian();
}
float min = Floats.min(values);
min = (float) (min < 0 ? 1.02 : .98) * min;
float max = Floats.max(values);
max = (float) (max < 0 ? .98 : 1.02) * max;
final float stride = (max - min) / numBuckets;
final float[] breaks = new float[numBreaks];
for (int i = 0; i < numBreaks; i++) {
breaks[i] = min + stride * i;
}
Histogram h = new Histogram(breaks);
for (float v : values) {
h.offer(v);
}
double[] hcounts = h.asVisual().counts;
ApproximateHistogram ah1 = new ApproximateHistogram(resolution);
ApproximateHistogram ah2 = new ApproximateHistogram(combinedResolution);
ApproximateHistogram tmp = new ApproximateHistogram(resolution);
for (int i = 0; i < numValues; ++i) {
tmp.offer(values[i]);
if ((i + 1) % numPerHist == 0) {
ah1.fold(tmp);
ah2.foldRule(tmp, null, null);
tmp = new ApproximateHistogram(resolution);
}
}
double[] ahcounts1 = ah1.toHistogram(breaks).getCounts();
double[] ahcounts2 = ah2.toHistogram(breaks).getCounts();
float err1 = 0;
float err2 = 0;
for (int j = 0; j < hcounts.length; j++) {
err1 += Math.abs((hcounts[j] - ahcounts1[j]) / numValues);
err2 += Math.abs((hcounts[j] - ahcounts2[j]) / numValues);
}
if (debug) {
float sum = 0;
for (double v : hcounts) {
sum += v;
}
System.out.println("Exact Histogram Sum:");
System.out.println(sum);
sum = 0;
for (double v : ahcounts1) {
sum += v;
}
System.out.println("Approximate Histogram Sum:");
System.out.println(sum);
sum = 0;
for (double v : ahcounts2) {
sum += v;
}
System.out.println("Approximate Histogram Rule Fold Sum:");
System.out.println(sum);
System.out.println("Exact Histogram:");
System.out.println(h.asVisual());
System.out.println("Approximate Histogram:");
System.out.println(ah1.toHistogram(breaks));
System.out.println("Approximate Histogram Rule Fold:");
System.out.println(ah2.toHistogram(breaks));
System.out.format("Error for approximate histogram: %s \n", err1);
System.out.format("Error for approximate histogram, ruleFold: %s \n", err2);
System.out.format("Error ratio for AHRF: %s \n", err2 / err1);
}
return new float[]{err1, err2, err2 / err1};
}
}

View File

@ -0,0 +1,65 @@
/*
* 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.aggregation.histogram;
import io.druid.query.aggregation.TestFloatColumnSelector;
import org.junit.Assert;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
public class ApproximateHistogramPostAggregatorTest
{
static final float[] VALUES = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10};
protected ApproximateHistogram buildHistogram(int size, float[] values)
{
ApproximateHistogram h = new ApproximateHistogram(size);
for (float v : values) {
h.offer(v);
}
return h;
}
@Test
public void testCompute()
{
ApproximateHistogram ah = buildHistogram(10, VALUES);
final TestFloatColumnSelector selector = new TestFloatColumnSelector(VALUES);
ApproximateHistogramAggregator agg = new ApproximateHistogramAggregator("price", selector, 10, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY);
for (int i = 0; i < VALUES.length; i++) {
agg.aggregate();
selector.increment();
}
Map<String, Object> metricValues = new HashMap<String, Object>();
metricValues.put(agg.getName(), agg.get());
ApproximateHistogramPostAggregator approximateHistogramPostAggregator = new EqualBucketsPostAggregator(
"approxHist",
"price",
5
);
Assert.assertEquals(ah.toHistogram(5), approximateHistogramPostAggregator.compute(metricValues));
}
}

View File

@ -0,0 +1,247 @@
/*
* 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.aggregation.histogram;
import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.collections.StupidPool;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
import io.druid.query.TestQueryRunners;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.MaxAggregatorFactory;
import io.druid.query.aggregation.MinAggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNQueryBuilder;
import io.druid.query.topn.TopNQueryConfig;
import io.druid.query.topn.TopNQueryQueryToolChest;
import io.druid.query.topn.TopNQueryRunnerFactory;
import io.druid.query.topn.TopNResultValue;
import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
@RunWith(Parameterized.class)
public class ApproximateHistogramQueryTest
{
private final QueryRunner runner;
public ApproximateHistogramQueryTest(
QueryRunner runner
)
{
this.runner = runner;
}
@Parameterized.Parameters
public static Collection<?> constructorFeeder() throws IOException
{
List<Object> retVal = Lists.newArrayList();
retVal.addAll(
QueryRunnerTestHelper.makeQueryRunners(
new TopNQueryRunnerFactory(
TestQueryRunners.getPool(),
new TopNQueryQueryToolChest(new TopNQueryConfig()),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
);
retVal.addAll(
QueryRunnerTestHelper.makeQueryRunners(
new TopNQueryRunnerFactory(
new StupidPool<ByteBuffer>(
new Supplier<ByteBuffer>()
{
@Override
public ByteBuffer get()
{
return ByteBuffer.allocate(2000);
}
}
),
new TopNQueryQueryToolChest(new TopNQueryConfig()),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
);
return retVal;
}
@Test
public void testTopNWithApproximateHistogramAgg()
{
ApproximateHistogramAggregatorFactory factory = new ApproximateHistogramAggregatorFactory(
"apphisto",
"index",
10,
5,
Float.NEGATIVE_INFINITY,
Float.POSITIVE_INFINITY
);
TopNQuery query = new TopNQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.dimension(QueryRunnerTestHelper.providerDimension)
.metric(QueryRunnerTestHelper.dependentPostAggMetric)
.threshold(4)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.aggregators(
Lists.<AggregatorFactory>newArrayList(
Iterables.concat(
QueryRunnerTestHelper.commonAggregators,
Lists.newArrayList(
new MaxAggregatorFactory("maxIndex", "index"),
new MinAggregatorFactory("minIndex", "index"),
factory
)
)
)
)
.postAggregators(
Arrays.<PostAggregator>asList(
QueryRunnerTestHelper.addRowsIndexConstant,
QueryRunnerTestHelper.dependentPostAgg,
new QuantilePostAggregator("quantile", "apphisto", 0.5f)
)
)
.build();
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
new Result<TopNResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new TopNResultValue(
Arrays.<Map<String, Object>>asList(
ImmutableMap.<String, Object>builder()
.put(QueryRunnerTestHelper.providerDimension, "total_market")
.put("rows", 186L)
.put("index", 215679.82879638672D)
.put("addRowsIndexConstant", 215866.82879638672D)
.put(QueryRunnerTestHelper.dependentPostAggMetric, 216053.82879638672D)
.put("uniques", QueryRunnerTestHelper.UNIQUES_2)
.put("maxIndex", 1743.9217529296875D)
.put("minIndex", 792.3260498046875D)
.put("quantile", 1085.6775f)
.put(
"apphisto",
new Histogram(
new float[]{
554.4271240234375f,
792.3260498046875f,
1030.2249755859375f,
1268.1239013671875f,
1506.0228271484375f,
1743.9217529296875f
},
new double[]{
0.0D,
39.42073059082031D,
103.29110717773438D,
34.93659591674805D,
8.351564407348633D
}
)
)
.build(),
ImmutableMap.<String, Object>builder()
.put(QueryRunnerTestHelper.providerDimension, "upfront")
.put("rows", 186L)
.put("index", 192046.1060180664D)
.put("addRowsIndexConstant", 192233.1060180664D)
.put(QueryRunnerTestHelper.dependentPostAggMetric, 192420.1060180664D)
.put("uniques", QueryRunnerTestHelper.UNIQUES_2)
.put("maxIndex", 1870.06103515625D)
.put("minIndex", 545.9906005859375D)
.put("quantile", 880.9881f)
.put(
"apphisto",
new Histogram(
new float[]{
214.97299194335938f,
545.9906005859375f,
877.0081787109375f,
1208.0257568359375f,
1539.0433349609375f,
1870.06103515625f
},
new double[]{
0.0D,
67.53287506103516D,
72.22068786621094D,
31.984678268432617D,
14.261756896972656D
}
)
)
.build(),
ImmutableMap.<String, Object>builder()
.put(QueryRunnerTestHelper.providerDimension, "spot")
.put("rows", 837L)
.put("index", 95606.57232284546D)
.put("addRowsIndexConstant", 96444.57232284546D)
.put(QueryRunnerTestHelper.dependentPostAggMetric, 97282.57232284546D)
.put("uniques", QueryRunnerTestHelper.UNIQUES_9)
.put("maxIndex", 277.2735290527344D)
.put("minIndex", 59.02102279663086D)
.put("quantile", 101.78856f)
.put(
"apphisto",
new Histogram(
new float[]{
4.457897186279297f,
59.02102279663086f,
113.58415222167969f,
168.14727783203125f,
222.7104034423828f,
277.2735290527344f
},
new double[]{
0.0D,
462.4309997558594D,
357.5404968261719D,
15.022850036621094D,
2.0056631565093994D
}
)
)
.build()
)
)
)
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query));
}
}

View File

@ -0,0 +1,588 @@
/*
* 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.aggregation.histogram;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import org.junit.Assert;
import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.Iterator;
import java.util.List;
import java.util.Random;
public class ApproximateHistogramTest
{
static final float[] VALUES = {23, 19, 10, 16, 36, 2, 9, 32, 30, 45};
static final float[] VALUES2 = {23, 19, 10, 16, 36, 2, 1, 9, 32, 30, 45, 46};
static final float[] VALUES3 = {
20, 16, 19, 27, 17, 20, 18, 20, 28, 14, 17, 21, 20, 21, 10, 25, 23, 17, 21, 18,
14, 20, 18, 12, 19, 20, 23, 25, 15, 22, 14, 17, 15, 23, 23, 15, 27, 20, 17, 15
};
static final float[] VALUES4 = {
27.489f, 3.085f, 3.722f, 66.875f, 30.998f, -8.193f, 5.395f, 5.109f, 10.944f, 54.75f,
14.092f, 15.604f, 52.856f, 66.034f, 22.004f, -14.682f, -50.985f, 2.872f, 61.013f,
-21.766f, 19.172f, 62.882f, 33.537f, 21.081f, 67.115f, 44.789f, 64.1f, 20.911f,
-6.553f, 2.178f
};
static final float[] VALUES5 = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10};
static final float[] VALUES6 = {1f, 1.5f, 2f, 2.5f, 3f, 3.5f, 4f, 4.5f, 5f, 5.5f, 6f, 6.5f, 7f, 7.5f, 8f, 8.5f, 9f, 9.5f, 10f};
protected ApproximateHistogram buildHistogram(int size, float[] values)
{
ApproximateHistogram h = new ApproximateHistogram(size);
for (float v : values) {
h.offer(v);
}
return h;
}
protected ApproximateHistogram buildHistogram(int size, float[] values, float lowerLimit, float upperLimit)
{
ApproximateHistogram h = new ApproximateHistogram(size, lowerLimit, upperLimit);
for (float v : values) {
h.offer(v);
}
return h;
}
@Test
public void testOffer() throws Exception
{
ApproximateHistogram h = buildHistogram(5, VALUES);
// (2, 1), (9.5, 2), (19.33, 3), (32.67, 3), (45, 1)
Assert.assertArrayEquals(
"final bin positions match expected positions",
new float[]{2, 9.5f, 19.33f, 32.67f, 45f}, h.positions(), 0.1f
);
Assert.assertArrayEquals(
"final bin positions match expected positions",
new long[]{1, 2, 3, 3, 1}, h.bins()
);
Assert.assertEquals("min value matches expexted min", 2, h.min(), 0);
Assert.assertEquals("max value matches expexted max", 45, h.max(), 0);
Assert.assertEquals("bin count matches expected bin count", 5, h.binCount());
}
@Test
public void testFold()
{
ApproximateHistogram merged = new ApproximateHistogram(0);
ApproximateHistogram mergedFast = new ApproximateHistogram(0);
ApproximateHistogram h1 = new ApproximateHistogram(5);
ApproximateHistogram h2 = new ApproximateHistogram(10);
for (int i = 0; i < 5; ++i) {
h1.offer(VALUES[i]);
}
for (int i = 5; i < VALUES.length; ++i) {
h2.offer(VALUES[i]);
}
merged.fold(h1);
merged.fold(h2);
mergedFast.foldFast(h1);
mergedFast.foldFast(h2);
Assert.assertArrayEquals(
"final bin positions match expected positions",
new float[]{2, 9.5f, 19.33f, 32.67f, 45f}, merged.positions(), 0.1f
);
Assert.assertArrayEquals(
"final bin positions match expected positions",
new float[]{11.2f, 30.25f, 45f}, mergedFast.positions(), 0.1f
);
Assert.assertArrayEquals(
"final bin counts match expected counts",
new long[]{1, 2, 3, 3, 1}, merged.bins()
);
Assert.assertArrayEquals(
"final bin counts match expected counts",
new long[]{5, 4, 1}, mergedFast.bins()
);
Assert.assertEquals("merged max matches expected value", 45f, merged.max(), 0.1f);
Assert.assertEquals("mergedfast max matches expected value", 45f, mergedFast.max(), 0.1f);
Assert.assertEquals("merged min matches expected value", 2f, merged.min(), 0.1f);
Assert.assertEquals("mergedfast min matches expected value", 2f, mergedFast.min(), 0.1f);
// fold where merged bincount is less than total bincount
ApproximateHistogram a = buildHistogram(10, new float[]{1, 2, 3, 4, 5, 6});
ApproximateHistogram aFast = buildHistogram(10, new float[]{1, 2, 3, 4, 5, 6});
ApproximateHistogram b = buildHistogram(5, new float[]{3, 4, 5, 6});
a.fold(b);
aFast.foldFast(b);
Assert.assertEquals(
new ApproximateHistogram(
6,
new float[]{1, 2, 3, 4, 5, 6, 0, 0, 0, 0},
new long[]{1, 1, 2, 2, 2, 2, 0, 0, 0, 0},
1, 6
), a
);
Assert.assertEquals(
new ApproximateHistogram(
6,
new float[]{1, 2, 3, 4, 5, 6, 0, 0, 0, 0},
new long[]{1, 1, 2, 2, 2, 2, 0, 0, 0, 0},
1, 6
), aFast
);
ApproximateHistogram h3 = new ApproximateHistogram(10);
ApproximateHistogram h4 = new ApproximateHistogram(10);
for (float v : VALUES3) {
h3.offer(v);
}
for (float v : VALUES4) {
h4.offer(v);
}
h3.fold(h4);
Assert.assertArrayEquals(
"final bin positions match expected positions",
new float[]{-50.98f, -21.77f, -9.81f, 3.73f, 13.72f, 20.1f, 29f, 44.79f, 53.8f, 64.67f},
h3.positions(), 0.1f
);
Assert.assertArrayEquals(
"final bin counts match expected counts",
new long[]{1, 1, 3, 6, 12, 32, 6, 1, 2, 6}, h3.bins()
);
}
@Test
public void testFoldNothing() throws Exception
{
ApproximateHistogram h1 = new ApproximateHistogram(10);
ApproximateHistogram h2 = new ApproximateHistogram(10);
h1.fold(h2);
h1.foldFast(h2);
}
@Test
public void testFoldNothing2() throws Exception
{
ApproximateHistogram h1 = new ApproximateHistogram(10);
ApproximateHistogram h1Fast = new ApproximateHistogram(10);
ApproximateHistogram h2 = new ApproximateHistogram(10);
ApproximateHistogram h3 = new ApproximateHistogram(10);
ApproximateHistogram h4 = new ApproximateHistogram(10);
ApproximateHistogram h4Fast = new ApproximateHistogram(10);
for (float v : VALUES3) {
h3.offer(v);
h4.offer(v);
h4Fast.offer(v);
}
h1.fold(h3);
h4.fold(h2);
h1Fast.foldFast(h3);
h4Fast.foldFast(h2);
Assert.assertEquals(h3, h1);
Assert.assertEquals(h4, h3);
Assert.assertEquals(h3, h1Fast);
Assert.assertEquals(h3, h4Fast);
}
//@Test
public void testFoldSpeed()
{
final int combinedHistSize = 200;
final int histSize = 50;
final int numRand = 10000;
ApproximateHistogram h = new ApproximateHistogram(combinedHistSize);
Random rand = new Random(0);
//for(int i = 0; i < 200; ++i) h.offer((float)(rand.nextGaussian() * 50.0));
long tFold = 0;
int count = 5000000;
Float[] randNums = new Float[numRand];
for (int i = 0; i < numRand; i++) {
randNums[i] = (float) rand.nextGaussian();
}
List<ApproximateHistogram> randHist = Lists.newLinkedList();
Iterator<ApproximateHistogram> it = Iterators.cycle(randHist);
for(int k = 0; k < numRand; ++k) {
ApproximateHistogram tmp = new ApproximateHistogram(histSize);
for (int i = 0; i < 20; ++i) {
tmp.offer((float) (rand.nextGaussian() + (double)k));
}
randHist.add(tmp);
}
float[] mergeBufferP = new float[combinedHistSize * 2];
long[] mergeBufferB = new long[combinedHistSize * 2];
float[] mergeBufferD = new float[combinedHistSize * 2];
for (int i = 0; i < count; ++i) {
ApproximateHistogram tmp = it.next();
long t0 = System.nanoTime();
//h.fold(tmp, mergeBufferP, mergeBufferB, mergeBufferD);
h.foldFast(tmp, mergeBufferP, mergeBufferB);
tFold += System.nanoTime() - t0;
}
System.out.println(String.format("Average folds per second : %f", (double) count / (double) tFold * 1e9));
}
@Test
public void testSum()
{
ApproximateHistogram h = buildHistogram(5, VALUES);
Assert.assertEquals(0.0f, h.sum(0), 0.01);
Assert.assertEquals(1.0f, h.sum(2), 0.01);
Assert.assertEquals(1.16f, h.sum(5), 0.01);
Assert.assertEquals(3.28f, h.sum(15), 0.01);
Assert.assertEquals(VALUES.length, h.sum(45), 0.01);
Assert.assertEquals(VALUES.length, h.sum(46), 0.01);
ApproximateHistogram h2 = buildHistogram(5, VALUES2);
Assert.assertEquals(0.0f, h2.sum(0), 0.01);
Assert.assertEquals(0.0f, h2.sum(1f), 0.01);
Assert.assertEquals(1.0f, h2.sum(1.5f), 0.01);
Assert.assertEquals(1.125f, h2.sum(2f), 0.001);
Assert.assertEquals(2.0625f, h2.sum(5.75f), 0.001);
Assert.assertEquals(3.0f, h2.sum(9.5f), 0.01);
Assert.assertEquals(11.0f, h2.sum(45.5f), 0.01);
Assert.assertEquals(12.0f, h2.sum(46f), 0.01);
Assert.assertEquals(12.0f, h2.sum(47f), 0.01);
}
@Test
public void testSerializeCompact()
{
ApproximateHistogram h = buildHistogram(5, VALUES);
Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes()));
ApproximateHistogram h2 = new ApproximateHistogram(50).fold(h);
Assert.assertEquals(h2, ApproximateHistogram.fromBytes(h2.toBytes()));
}
@Test
public void testSerializeDense()
{
ApproximateHistogram h = buildHistogram(5, VALUES);
ByteBuffer buf = ByteBuffer.allocate(h.getDenseStorageSize());
h.toBytesDense(buf);
Assert.assertEquals(h, ApproximateHistogram.fromBytes(buf.array()));
}
@Test
public void testSerializeSparse()
{
ApproximateHistogram h = buildHistogram(5, VALUES);
ByteBuffer buf = ByteBuffer.allocate(h.getSparseStorageSize());
h.toBytesSparse(buf);
Assert.assertEquals(h, ApproximateHistogram.fromBytes(buf.array()));
}
@Test
public void testSerializeCompactExact()
{
ApproximateHistogram h = buildHistogram(50, new float[]{1f, 2f, 3f, 4f, 5f});
Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes()));
h = buildHistogram(5, new float[]{1f, 2f, 3f});
Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes()));
h = new ApproximateHistogram(40).fold(h);
Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes()));
}
@Test
public void testSerializeEmpty()
{
ApproximateHistogram h = new ApproximateHistogram(50);
Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes()));
}
@Test
public void testQuantileSmaller()
{
ApproximateHistogram h = buildHistogram(20, VALUES5);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{5f},
h.getQuantiles(new float[]{.5f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{3.33f, 6.67f},
h.getQuantiles(new float[]{.333f, .666f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{2.5f, 5f, 7.5f},
h.getQuantiles(new float[]{.25f, .5f, .75f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{2f, 4f, 6f, 8f},
h.getQuantiles(new float[]{.2f, .4f, .6f, .8f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{1f, 2f, 3f, 4f, 5f, 6f, 7f, 8f, 9f},
h.getQuantiles(new float[]{.1f, .2f, .3f, .4f, .5f, .6f, .7f, .8f, .9f}), 0.1f
);
}
@Test
public void testQuantileEqualSize()
{
ApproximateHistogram h = buildHistogram(10, VALUES5);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{5f},
h.getQuantiles(new float[]{.5f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{3.33f, 6.67f},
h.getQuantiles(new float[]{.333f, .666f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{2.5f, 5f, 7.5f},
h.getQuantiles(new float[]{.25f, .5f, .75f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{2f, 4f, 6f, 8f},
h.getQuantiles(new float[]{.2f, .4f, .6f, .8f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{1f, 2f, 3f, 4f, 5f, 6f, 7f, 8f, 9f},
h.getQuantiles(new float[]{.1f, .2f, .3f, .4f, .5f, .6f, .7f, .8f, .9f}), 0.1f
);
}
@Test
public void testQuantileBigger()
{
ApproximateHistogram h = buildHistogram(5, VALUES5);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{4.5f},
h.getQuantiles(new float[]{.5f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{2.83f, 6.17f},
h.getQuantiles(new float[]{.333f, .666f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{2f, 4.5f, 7f},
h.getQuantiles(new float[]{.25f, .5f, .75f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{1.5f, 3.5f, 5.5f, 7.5f},
h.getQuantiles(new float[]{.2f, .4f, .6f, .8f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{1f, 1.5f, 2.5f, 3.5f, 4.5f, 5.5f, 6.5f, 7.5f, 8.5f},
h.getQuantiles(new float[]{.1f, .2f, .3f, .4f, .5f, .6f, .7f, .8f, .9f}), 0.1f
);
}
@Test
public void testQuantileBigger2()
{
float[] thousand = new float[1000];
for (int i = 1; i <= 1000; ++i) {
thousand[i - 1] = i;
}
ApproximateHistogram h = buildHistogram(100, thousand);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{493.5f},
h.getQuantiles(new float[]{.5f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{327.5f, 662f},
h.getQuantiles(new float[]{.333f, .666f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{244.5f, 493.5f, 746f},
h.getQuantiles(new float[]{.25f, .5f, .75f}), 0.1f
);
Assert.assertArrayEquals(
"expected quantiles match actual quantiles",
new float[]{96.5f, 196.53f, 294.5f, 395.5f, 493.5f, 597f, 696f, 795f, 895.25f},
h.getQuantiles(new float[]{.1f, .2f, .3f, .4f, .5f, .6f, .7f, .8f, .9f}), 0.1f
);
}
@Test
public void testLimitSum()
{
final float lowerLimit = 0f;
final float upperLimit = 10f;
ApproximateHistogram h = buildHistogram(15, VALUES6, lowerLimit, upperLimit);
for (int i = 1; i <= 20; ++i) {
ApproximateHistogram hLow = new ApproximateHistogram(5);
ApproximateHistogram hHigh = new ApproximateHistogram(5);
hLow.offer(lowerLimit - i);
hHigh.offer(upperLimit + i);
h.foldFast(hLow);
h.foldFast(hHigh);
}
Assert.assertEquals(20f, h.sum(lowerLimit), .7f);
Assert.assertEquals(VALUES6.length + 20f, h.sum(upperLimit), 0.01);
}
@Test
public void testBuckets()
{
final float[] values = new float[]{-5f, .01f, .02f, .06f, .12f, 1f, 2f};
ApproximateHistogram h = buildHistogram(50, values, 0f, 1f);
Histogram h2 = h.toHistogram(.05f, 0f);
Assert.assertArrayEquals(
"expected counts match actual counts",
new double[]{1f, 2f, 1f, 1f, 0f, 1f, 1f},
h2.getCounts(), 0.1f
);
Assert.assertArrayEquals(
"expected breaks match actual breaks",
new double[]{-5.05f, 0f, .05f, .1f, .15f, .95f, 1f, 2f},
h2.getBreaks(), 0.1f
);
}
@Test
public void testBuckets2()
{
final float[] values = new float[]{-5f, .01f, .02f, .06f, .12f, .94f, 1f, 2f};
ApproximateHistogram h = buildHistogram(50, values, 0f, 1f);
Histogram h2 = h.toHistogram(.05f, 0f);
Assert.assertArrayEquals(
"expected counts match actual counts",
new double[]{1f, 2f, 1f, 1f, 0f, 1f, 1f, 1f},
h2.getCounts(), 0.1f
);
Assert.assertArrayEquals(
"expected breaks match actual breaks",
new double[]{-5.05f, 0f, .05f, .1f, .15f, .9f, .95f, 1f, 2.05f},
h2.getBreaks(), 0.1f
);
}
@Test
public void testBuckets3()
{
final float[] values = new float[]{0f, 0f, .02f, .06f, .12f, .94f};
ApproximateHistogram h = buildHistogram(50, values, 0f, 1f);
Histogram h2 = h.toHistogram(1f, 0f);
Assert.assertArrayEquals(
"expected counts match actual counts",
new double[]{2f, 4f},
h2.getCounts(), 0.1f
);
Assert.assertArrayEquals(
"expected breaks match actual breaks",
new double[]{-1f, 0f, 1f},
h2.getBreaks(), 0.1f
);
}
@Test
public void testBuckets4()
{
final float[] values = new float[]{0f, 0f, 0.01f, 0.51f, 0.6f,0.8f};
ApproximateHistogram h = buildHistogram(50, values, 0.5f,1f);
Histogram h3 = h.toHistogram(0.2f,0);
Assert.assertArrayEquals(
"Expected counts match actual counts",
new double[]{3f,2f,1f},
h3.getCounts(),
0.1f
);
Assert.assertArrayEquals(
"expected breaks match actual breaks",
new double[]{-0.2f,0.5f,0.7f,0.9f},
h3.getBreaks(), 0.1f
);
}
@Test public void testBuckets5()
{
final float[] values = new float[]{0.1f,0.5f,0.6f};
ApproximateHistogram h = buildHistogram(50, values, 0f,1f);
Histogram h4 = h.toHistogram(0.5f,0);
Assert.assertArrayEquals(
"Expected counts match actual counts",
new double[]{2,1},
h4.getCounts(),
0.1f
);
Assert.assertArrayEquals(
"Expected breaks match actual breaks",
new double[]{0f,0.5f,1f},
h4.getBreaks(),
0.1f
);
}
@Test public void testEmptyHistogram() {
ApproximateHistogram h = new ApproximateHistogram(50);
Assert.assertArrayEquals(
new float[]{Float.NaN, Float.NaN},
h.getQuantiles(new float[]{0.8f, 0.9f}),
1e-9f
);
}
}

View File

@ -0,0 +1,81 @@
/*
* 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.aggregation.histogram;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.jackson.DefaultObjectMapper;
import org.hamcrest.CoreMatchers;
import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
import java.util.LinkedHashMap;
public class QuantilesTest
{
@Test
public void testSerialization() throws Exception
{
ObjectMapper mapper = new DefaultObjectMapper();
float[] probabilities = new float[]{0.25f, 0.5f, 0.75f};
float[] quantiles = new float[]{0.25f, 0.5f, 0.75f};
float min = 0f;
float max = 4f;
String theString = mapper.writeValueAsString(
new Quantiles(probabilities, quantiles, min, max)
);
Object theObject = mapper.readValue(theString, Object.class);
Assert.assertThat(theObject, CoreMatchers.instanceOf(LinkedHashMap.class));
LinkedHashMap theMap = (LinkedHashMap) theObject;
ArrayList theProbabilities = (ArrayList<Float>) theMap.get("probabilities");
Assert.assertEquals(probabilities.length, theProbabilities.size());
for (int i = 0; i < theProbabilities.size(); ++i) {
Assert.assertEquals(probabilities[i], ((Number) theProbabilities.get(i)).floatValue(), 0.0001f);
}
ArrayList theQuantiles = (ArrayList<Float>) theMap.get("quantiles");
Assert.assertEquals(quantiles.length, theQuantiles.size());
for (int i = 0; i < theQuantiles.size(); ++i) {
Assert.assertEquals(quantiles[i], ((Number) theQuantiles.get(i)).floatValue(), 0.0001f);
}
Assert.assertEquals(
"serialized min. matches expected min.",
min,
((Number) theMap.get("min")).floatValue(),
0.0001f
);
Assert.assertEquals(
"serialized max. matches expected max.",
max,
((Number) theMap.get("max")).floatValue(),
0.0001f
);
}
}

View File

View File

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

View File

@ -44,7 +44,9 @@ import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.path.PathSpec;
import io.druid.guice.GuiceInjectors;
import io.druid.initialization.Initialization;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.indexing.granularity.GranularitySpec;
import io.druid.server.DruidNode;
import io.druid.timeline.DataSegment;
@ -81,7 +83,7 @@ public class HadoopDruidIndexerConfig
static {
injector = Initialization.makeInjectorWithModules(
Initialization.makeStartupInjector(),
GuiceInjectors.makeStartupInjector(),
ImmutableList.<Object>of(
new Module()
{
@ -166,12 +168,14 @@ public class HadoopDruidIndexerConfig
private volatile HadoopIngestionSpec schema;
private volatile PathSpec pathSpec;
private volatile ColumnConfig columnConfig;
@JsonCreator
public HadoopDruidIndexerConfig(
final @JsonProperty("schema") HadoopIngestionSpec schema
)
{
this.columnConfig = columnConfig;
this.schema = schema;
this.pathSpec = jsonMapper.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class);
}
@ -182,6 +186,11 @@ public class HadoopDruidIndexerConfig
return schema;
}
public ColumnConfig getColumnConfig()
{
return columnConfig;
}
public String getDataSource()
{
return schema.getDataSchema().getDataSource();

View File

@ -31,6 +31,7 @@ import com.google.common.io.Closeables;
import com.google.common.primitives.Longs;
import com.metamx.common.IAE;
import com.metamx.common.ISE;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.logger.Logger;
import io.druid.data.input.InputRow;
import io.druid.data.input.impl.StringInputRowParser;
@ -427,7 +428,7 @@ public class IndexGeneratorJob implements Jobby
if (caughtException == null) {
Closeables.close(out, false);
} else {
Closeables.closeQuietly(out);
CloseQuietly.close(out);
throw Throwables.propagate(caughtException);
}
}
@ -607,7 +608,7 @@ public class IndexGeneratorJob implements Jobby
}
}
finally {
Closeables.closeQuietly(in);
CloseQuietly.close(in);
}
out.closeEntry();
context.progress();

View File

@ -45,7 +45,6 @@ public class HadoopDruidIndexerConfigTest
}
}
@Test
public void shouldMakeHDFSCompliantSegmentOutputPath()
{

View File

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

View File

@ -49,7 +49,6 @@ import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutorService;
/**
@ -187,7 +186,8 @@ public class TaskToolbox
return retVal;
}
public void pushSegments(Iterable<DataSegment> segments) throws IOException {
public void pushSegments(Iterable<DataSegment> segments) throws IOException
{
// Request segment pushes for each set
final Multimap<Interval, DataSegment> segmentMultimap = Multimaps.index(
segments,

View File

@ -29,6 +29,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.common.task.Task;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.loading.DataSegmentArchiver;
import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.DataSegmentMover;

View File

@ -21,11 +21,17 @@ package io.druid.indexing.common.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import java.io.File;
import java.util.List;
public class TaskConfig
{
public static List<String> DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of(
"org.apache.hadoop:hadoop-client:2.3.0"
);
@JsonProperty
private final String baseDir;
@ -38,40 +44,57 @@ public class TaskConfig
@JsonProperty
private final int defaultRowFlushBoundary;
@JsonProperty
private final List<String> defaultHadoopCoordinates;
@JsonCreator
public TaskConfig(
@JsonProperty("baseDir") String baseDir,
@JsonProperty("baseTaskDir") String baseTaskDir,
@JsonProperty("hadoopWorkingPath") String hadoopWorkingPath,
@JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary
@JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary,
@JsonProperty("defaultHadoopCoordinates") List<String> defaultHadoopCoordinates
)
{
this.baseDir = baseDir == null ? "/tmp" : baseDir;
this.baseTaskDir = new File(defaultDir(baseTaskDir, "persistent/task"));
this.hadoopWorkingPath = defaultDir(hadoopWorkingPath, "druid-indexing");
this.defaultRowFlushBoundary = defaultRowFlushBoundary == null ? 500000 : defaultRowFlushBoundary;
this.defaultHadoopCoordinates = defaultHadoopCoordinates == null
? DEFAULT_DEFAULT_HADOOP_COORDINATES
: defaultHadoopCoordinates;
}
@JsonProperty
public String getBaseDir()
{
return baseDir;
}
@JsonProperty
public File getBaseTaskDir()
{
return baseTaskDir;
}
@JsonProperty
public String getHadoopWorkingPath()
{
return hadoopWorkingPath;
}
@JsonProperty
public int getDefaultRowFlushBoundary()
{
return defaultRowFlushBoundary;
}
@JsonProperty
public List<String> getDefaultHadoopCoordinates()
{
return defaultHadoopCoordinates;
}
private String defaultDir(String configParameter, final String defaultVal)
{
if (configParameter == null) {
@ -80,4 +103,4 @@ public class TaskConfig
return configParameter;
}
}
}

View File

@ -27,6 +27,7 @@ import com.google.api.client.util.Lists;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.metamx.common.logger.Logger;
import io.druid.common.utils.JodaUtils;
@ -41,8 +42,9 @@ import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockAcquireAction;
import io.druid.indexing.common.actions.LockTryAcquireAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.guice.GuiceInjectors;
import io.druid.initialization.Initialization;
import io.druid.server.initialization.ExtensionsConfig;
import io.druid.guice.ExtensionsConfig;
import io.druid.timeline.DataSegment;
import io.tesla.aether.internal.DefaultTeslaAether;
import org.joda.time.DateTime;
@ -62,11 +64,9 @@ public class HadoopIndexTask extends AbstractTask
private static final ExtensionsConfig extensionsConfig;
static {
extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class);
extensionsConfig = GuiceInjectors.makeStartupInjector().getInstance(ExtensionsConfig.class);
}
public static String DEFAULT_HADOOP_COORDINATES = "org.apache.hadoop:hadoop-client:2.3.0";
private static String getTheDataSource(HadoopIngestionSpec spec, HadoopIngestionSpec config)
{
if (spec != null) {
@ -115,9 +115,14 @@ public class HadoopIndexTask extends AbstractTask
Preconditions.checkArgument(this.spec.getTuningConfig().getWorkingPath() == null, "workingPath must be absent");
Preconditions.checkArgument(this.spec.getIOConfig().getMetadataUpdateSpec() == null, "updaterJobSpec must be absent");
this.hadoopDependencyCoordinates = hadoopDependencyCoordinates == null ? Arrays.<String>asList(
hadoopCoordinates == null ? DEFAULT_HADOOP_COORDINATES : hadoopCoordinates
) : hadoopDependencyCoordinates;
if (hadoopDependencyCoordinates != null) {
this.hadoopDependencyCoordinates = hadoopDependencyCoordinates;
} else if (hadoopCoordinates != null) {
this.hadoopDependencyCoordinates = ImmutableList.of(hadoopCoordinates);
} else {
// Will be defaulted to something at runtime, based on taskConfig.
this.hadoopDependencyCoordinates = null;
}
}
@Override
@ -158,6 +163,10 @@ public class HadoopIndexTask extends AbstractTask
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
final List<String> finalHadoopDependencyCoordinates = hadoopDependencyCoordinates != null
? hadoopDependencyCoordinates
: toolbox.getConfig().getDefaultHadoopCoordinates();
final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig);
final List<URL> extensionURLs = Lists.newArrayList();
@ -174,7 +183,7 @@ public class HadoopIndexTask extends AbstractTask
final List<URL> driverURLs = Lists.newArrayList();
driverURLs.addAll(nonHadoopURLs);
// put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts
for (String hadoopDependencyCoordinate : hadoopDependencyCoordinates) {
for (String hadoopDependencyCoordinate : finalHadoopDependencyCoordinates) {
final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates(
aetherClient, hadoopDependencyCoordinate
);

View File

@ -19,14 +19,15 @@
package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.io.Closeables;
import com.metamx.common.Granularity;
import com.metamx.common.exception.FormattedException;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.parsers.ParseException;
import com.metamx.emitter.EmittingLogger;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
@ -43,9 +44,10 @@ import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.query.QueryToolChest;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeTuningConfig;
import io.druid.segment.indexing.RealtimeIOConfig;
import io.druid.segment.indexing.RealtimeTuningConfig;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.segment.realtime.FireDepartment;
import io.druid.segment.realtime.FireDepartmentConfig;
@ -353,7 +355,7 @@ public class RealtimeIndexTask extends AbstractTask
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
}
}
catch (FormattedException e) {
catch (ParseException e) {
log.warn(e, "unparseable line");
fireDepartment.getMetrics().incrementUnparseable();
}
@ -375,7 +377,7 @@ public class RealtimeIndexTask extends AbstractTask
log.makeAlert(e, "Failed to finish realtime task").emit();
}
finally {
Closeables.closeQuietly(firehose);
CloseQuietly.close(firehose);
toolbox.getMonitorScheduler().removeMonitor(metricsMonitor);
}
}

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Sets;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.logger.Logger;

View File

@ -79,6 +79,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
private final DruidNode node;
private final ListeningExecutorService exec;
private final ObjectMapper jsonMapper;
private final PortFinder portFinder;
private final Map<String, ForkingTaskRunnerWorkItem> tasks = Maps.newHashMap();
@ -97,6 +98,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
this.taskLogPusher = taskLogPusher;
this.jsonMapper = jsonMapper;
this.node = node;
this.portFinder = new PortFinder(config.getStartPort());
this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(workerConfig.getCapacity()));
}
@ -121,7 +123,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
final File attemptDir = new File(taskDir, attemptUUID);
final ProcessHolder processHolder;
final int childPort = portFinder.findUnusedPort();
try {
final Closer closer = Closer.create();
try {
@ -154,7 +156,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
final List<String> command = Lists.newArrayList();
final int childPort = findUnusedPort();
final String childHost = String.format("%s:%d", node.getHostNoPort(), childPort);
command.add(config.getJavaCommand());
@ -258,7 +259,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
taskWorkItem.processHolder.process.destroy();
}
}
portFinder.markPortUnused(childPort);
log.info("Removing temporary directory: %s", attemptDir);
FileUtils.deleteDirectory(attemptDir);
}

View File

@ -0,0 +1,94 @@
/*
* 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.indexing.overlord;
import com.google.common.collect.Sets;
import com.metamx.common.ISE;
import java.io.IOException;
import java.net.BindException;
import java.net.ServerSocket;
import java.util.Set;
public class PortFinder
{
private final Set<Integer> usedPorts = Sets.newHashSet();
private final int startPort;
public PortFinder(int startPort)
{
this.startPort = startPort;
}
private static boolean canBind(int portNum)
{
ServerSocket ss = null;
boolean isFree = false;
try {
ss = new ServerSocket(portNum);
isFree = true;
}
catch (BindException be) {
isFree = false; // port in use,
}
catch (IOException e) {
throw new RuntimeException(e);
}
finally {
if (ss != null) {
while (!ss.isClosed()) {
try {
ss.close();
}
catch (IOException e) {
// ignore
}
}
}
}
return isFree;
}
public synchronized int findUnusedPort()
{
int port = chooseNext(startPort);
while (!canBind(port)) {
port = chooseNext(port + 1);
}
usedPorts.add(port);
return port;
}
public synchronized void markPortUnused(int port)
{
usedPorts.remove(port);
}
private int chooseNext(int start)
{
for (int i = start; i < Integer.MAX_VALUE; i++) {
if (!usedPorts.contains(i)) {
return i;
}
}
throw new ISE("All ports are Used..");
}
}

View File

@ -585,7 +585,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
// Syncing state with Zookeeper - don't assign new tasks until the task we just assigned is actually running
// on a worker - this avoids overflowing a worker with tasks
Stopwatch timeoutStopwatch = new Stopwatch();
Stopwatch timeoutStopwatch = Stopwatch.createUnstarted();
timeoutStopwatch.start();
synchronized (statusLock) {
while (!isWorkerRunningTask(theWorker, task.getId())) {

View File

@ -29,19 +29,14 @@ import com.amazonaws.services.ec2.model.Reservation;
import com.amazonaws.services.ec2.model.RunInstancesRequest;
import com.amazonaws.services.ec2.model.RunInstancesResult;
import com.amazonaws.services.ec2.model.TerminateInstancesRequest;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
import com.google.inject.Inject;
import com.metamx.emitter.EmittingLogger;
import io.druid.guice.annotations.Json;
import io.druid.indexing.overlord.setup.EC2NodeData;
import io.druid.indexing.overlord.setup.GalaxyUserData;
import io.druid.indexing.overlord.setup.WorkerSetupData;
import org.apache.commons.codec.binary.Base64;
import javax.annotation.Nullable;
import java.util.List;
/**
@ -50,20 +45,17 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
{
private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class);
private final ObjectMapper jsonMapper;
private final AmazonEC2 amazonEC2Client;
private final SimpleResourceManagementConfig config;
private final Supplier<WorkerSetupData> workerSetupDataRef;
@Inject
public EC2AutoScalingStrategy(
@Json ObjectMapper jsonMapper,
AmazonEC2 amazonEC2Client,
SimpleResourceManagementConfig config,
Supplier<WorkerSetupData> workerSetupDataRef
)
{
this.jsonMapper = jsonMapper;
this.amazonEC2Client = amazonEC2Client;
this.config = config;
this.workerSetupDataRef = workerSetupDataRef;
@ -73,15 +65,21 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
public AutoScalingData provision()
{
try {
WorkerSetupData setupData = workerSetupDataRef.get();
EC2NodeData workerConfig = setupData.getNodeData();
final WorkerSetupData setupData = workerSetupDataRef.get();
final EC2NodeData workerConfig = setupData.getNodeData();
final String userDataBase64;
GalaxyUserData userData = setupData.getUserData();
if (config.getWorkerVersion() != null) {
userData = userData.withVersion(config.getWorkerVersion());
if (setupData.getUserData() == null) {
userDataBase64 = null;
} else {
if (config.getWorkerVersion() == null) {
userDataBase64 = setupData.getUserData().getUserDataBase64();
} else {
userDataBase64 = setupData.getUserData().withVersion(config.getWorkerVersion()).getUserDataBase64();
}
}
RunInstancesResult result = amazonEC2Client.runInstances(
final RunInstancesResult result = amazonEC2Client.runInstances(
new RunInstancesRequest(
workerConfig.getAmiId(),
workerConfig.getMinInstances(),
@ -91,16 +89,10 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
.withSecurityGroupIds(workerConfig.getSecurityGroupIds())
.withPlacement(new Placement(setupData.getAvailabilityZone()))
.withKeyName(workerConfig.getKeyName())
.withUserData(
Base64.encodeBase64String(
jsonMapper.writeValueAsBytes(
userData
)
)
)
.withUserData(userDataBase64)
);
List<String> instanceIds = Lists.transform(
final List<String> instanceIds = Lists.transform(
result.getReservation().getInstances(),
new Function<Instance, String>()
{

View File

@ -0,0 +1,42 @@
/*
* 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.indexing.overlord.setup;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
/**
* Represents any user data that may be needed to launch EC2 instances.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "impl", defaultImpl = GalaxyEC2UserData.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "galaxy", value = GalaxyEC2UserData.class),
@JsonSubTypes.Type(name = "string", value = StringEC2UserData.class)
})
public interface EC2UserData<T extends EC2UserData>
{
/**
* Return a copy of this instance with a different worker version. If no changes are needed (possibly because the
* user data does not depend on the worker version) then it is OK to return "this".
*/
public EC2UserData<T> withVersion(String version);
public String getUserDataBase64();
}

View File

@ -19,24 +19,32 @@
package io.druid.indexing.overlord.setup;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.api.client.repackaged.com.google.common.base.Throwables;
import io.druid.guice.annotations.Json;
import org.apache.commons.codec.binary.Base64;
/**
*/
public class GalaxyUserData
public class GalaxyEC2UserData implements EC2UserData<GalaxyEC2UserData>
{
public final String env;
public final String version;
public final String type;
private final ObjectMapper jsonMapper;
private final String env;
private final String version;
private final String type;
@JsonCreator
public GalaxyUserData(
public GalaxyEC2UserData(
@JacksonInject @Json ObjectMapper jsonMapper,
@JsonProperty("env") String env,
@JsonProperty("version") String version,
@JsonProperty("type") String type
)
{
this.jsonMapper = jsonMapper;
this.env = env;
this.version = version;
this.type = type;
@ -60,9 +68,21 @@ public class GalaxyUserData
return type;
}
public GalaxyUserData withVersion(String ver)
@Override
public GalaxyEC2UserData withVersion(String ver)
{
return new GalaxyUserData(env, ver, type);
return new GalaxyEC2UserData(jsonMapper, env, ver, type);
}
@Override
public String getUserDataBase64()
{
try {
return Base64.encodeBase64String(jsonMapper.writeValueAsBytes(this));
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override

View File

@ -0,0 +1,90 @@
/*
* 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.indexing.overlord.setup;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.api.client.util.Charsets;
import org.apache.commons.codec.binary.Base64;
public class StringEC2UserData implements EC2UserData<StringEC2UserData>
{
private final String data;
private final String versionReplacementString;
private final String version;
@JsonCreator
public StringEC2UserData(
@JsonProperty("data") String data,
@JsonProperty("versionReplacementString") String versionReplacementString,
@JsonProperty("version") String version
)
{
this.data = data;
this.versionReplacementString = versionReplacementString;
this.version = version;
}
@JsonProperty
public String getData()
{
return data;
}
@JsonProperty
public String getVersionReplacementString()
{
return versionReplacementString;
}
@JsonProperty
public String getVersion()
{
return version;
}
@Override
public StringEC2UserData withVersion(final String _version)
{
return new StringEC2UserData(data, versionReplacementString, _version);
}
@Override
public String getUserDataBase64()
{
final String finalData;
if (versionReplacementString != null && version != null) {
finalData = data.replace(versionReplacementString, version);
} else {
finalData = data;
}
return Base64.encodeBase64String(finalData.getBytes(Charsets.UTF_8));
}
@Override
public String toString()
{
return "StringEC2UserData{" +
"data='" + data + '\'' +
", versionReplacementString='" + versionReplacementString + '\'' +
", version='" + version + '\'' +
'}';
}
}

View File

@ -33,7 +33,7 @@ public class WorkerSetupData
private final int maxNumWorkers;
private final String availabilityZone;
private final EC2NodeData nodeData;
private final GalaxyUserData userData;
private final EC2UserData userData;
@JsonCreator
public WorkerSetupData(
@ -42,7 +42,7 @@ public class WorkerSetupData
@JsonProperty("maxNumWorkers") int maxNumWorkers,
@JsonProperty("availabilityZone") String availabilityZone,
@JsonProperty("nodeData") EC2NodeData nodeData,
@JsonProperty("userData") GalaxyUserData userData
@JsonProperty("userData") EC2UserData userData
)
{
this.minVersion = minVersion;
@ -84,7 +84,7 @@ public class WorkerSetupData
}
@JsonProperty
public GalaxyUserData getUserData()
public EC2UserData getUserData()
{
return userData;
}

View File

@ -19,19 +19,51 @@
package io.druid.indexing.common;
import com.fasterxml.jackson.databind.BeanProperty;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Stopwatch;
import com.metamx.common.ISE;
import io.druid.guice.ServerModule;
import io.druid.jackson.DefaultObjectMapper;
import java.util.List;
import java.util.concurrent.TimeUnit;
/**
*/
public class TestUtils
{
public static final ObjectMapper MAPPER = new DefaultObjectMapper();
static {
final List<? extends Module> list = new ServerModule().getJacksonModules();
for (Module module : list) {
MAPPER.registerModule(module);
}
MAPPER.setInjectableValues(
new InjectableValues()
{
@Override
public Object findInjectableValue(
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
)
{
if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) {
return TestUtils.MAPPER;
}
throw new ISE("No Injectable value found");
}
}
);
}
public static boolean conditionValid(IndexingServiceCondition condition)
{
try {
Stopwatch stopwatch = new Stopwatch();
Stopwatch stopwatch = Stopwatch.createUnstarted();
stopwatch.start();
while (!condition.isValid()) {
Thread.sleep(100);

View File

@ -48,6 +48,8 @@ import java.io.File;
public class TaskSerdeTest
{
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
@Test
public void testIndexTaskSerde() throws Exception
{
@ -68,7 +70,6 @@ public class TaskSerdeTest
-1
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) {
jsonMapper.registerModule(jacksonModule);
}
@ -102,7 +103,6 @@ public class TaskSerdeTest
)
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@ -131,7 +131,6 @@ public class TaskSerdeTest
new Interval("2010-01-01/P1D")
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@ -153,7 +152,6 @@ public class TaskSerdeTest
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@ -177,7 +175,6 @@ public class TaskSerdeTest
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@ -208,7 +205,6 @@ public class TaskSerdeTest
null
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@ -253,7 +249,6 @@ public class TaskSerdeTest
)
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@ -278,7 +273,6 @@ public class TaskSerdeTest
new Interval("2010-01-01/P1D")
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@ -303,7 +297,6 @@ public class TaskSerdeTest
new Interval("2010-01-01/P1D")
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@ -328,7 +321,6 @@ public class TaskSerdeTest
ImmutableMap.<String, Object>of("bucket", "hey", "baseKey", "what")
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@ -387,7 +379,6 @@ public class TaskSerdeTest
null
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.readValue(json, Task.class);

View File

@ -0,0 +1,34 @@
package io.druid.indexing.overlord;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.net.ServerSocket;
public class PortFinderTest
{
private final PortFinder finder = new PortFinder(1200);
@Test
public void testUsedPort() throws IOException
{
final int port1 = finder.findUnusedPort();
// verify that the port is free
ServerSocket socket1 = new ServerSocket(port1);
finder.markPortUnused(port1);
final int port2 = finder.findUnusedPort();
Assert.assertNotEquals("Used port is not reallocated", port1, port2);
// verify that port2 is free
ServerSocket socket2 = new ServerSocket(port2);
socket1.close();
// Now port1 should get recycled
Assert.assertEquals(port1, finder.findUnusedPort());
socket2.close();
finder.markPortUnused(port1);
finder.markPortUnused(port2);
}
}

View File

@ -43,6 +43,7 @@ import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.InputRowParser;
import io.druid.granularity.QueryGranularity;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskLock;
@ -135,7 +136,7 @@ public class TaskLifecycleTest
mdc = newMockMDC();
tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tl, mdc, newMockEmitter()));
tb = new TaskToolboxFactory(
new TaskConfig(tmp.toString(), null, null, 50000),
new TaskConfig(tmp.toString(), null, null, 50000, null),
tac,
newMockEmitter(),
new DataSegmentPusher()

View File

@ -0,0 +1,62 @@
/*
* 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.indexing.overlord;
import com.google.common.base.Charsets;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.overlord.setup.EC2UserData;
import io.druid.indexing.overlord.setup.GalaxyEC2UserData;
import io.druid.indexing.overlord.setup.StringEC2UserData;
import org.apache.commons.codec.binary.Base64;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
public class WorkerSetupDataTest
{
@Test
public void testGalaxyEC2UserDataSerde() throws IOException
{
final String json = "{\"env\":\"druid\",\"version\":null,\"type\":\"typical\"}";
final GalaxyEC2UserData userData = (GalaxyEC2UserData) TestUtils.MAPPER.readValue(json, EC2UserData.class);
Assert.assertEquals("druid", userData.getEnv());
Assert.assertEquals("typical", userData.getType());
Assert.assertNull(userData.getVersion());
Assert.assertEquals("1234", userData.withVersion("1234").getVersion());
}
@Test
public void testStringEC2UserDataSerde() throws IOException
{
final String json = "{\"impl\":\"string\",\"data\":\"hey :ver:\",\"versionReplacementString\":\":ver:\",\"version\":\"1234\"}";
final StringEC2UserData userData = (StringEC2UserData) TestUtils.MAPPER.readValue(json, EC2UserData.class);
Assert.assertEquals("hey :ver:", userData.getData());
Assert.assertEquals("1234", userData.getVersion());
Assert.assertEquals(
Base64.encodeBase64String("hey 1234".getBytes(Charsets.UTF_8)),
userData.getUserDataBase64()
);
Assert.assertEquals(
Base64.encodeBase64String("hey xyz".getBytes(Charsets.UTF_8)),
userData.withVersion("xyz").getUserDataBase64()
);
}
}

View File

@ -30,7 +30,7 @@ import com.amazonaws.services.ec2.model.TerminateInstancesRequest;
import com.google.common.collect.Lists;
import io.druid.common.guava.DSuppliers;
import io.druid.indexing.overlord.setup.EC2NodeData;
import io.druid.indexing.overlord.setup.GalaxyUserData;
import io.druid.indexing.overlord.setup.GalaxyEC2UserData;
import io.druid.indexing.overlord.setup.WorkerSetupData;
import io.druid.jackson.DefaultObjectMapper;
import org.easymock.EasyMock;
@ -75,7 +75,6 @@ public class EC2AutoScalingStrategyTest
.withPrivateIpAddress(IP);
strategy = new EC2AutoScalingStrategy(
new DefaultObjectMapper(),
amazonEC2Client,
new SimpleResourceManagementConfig().setWorkerPort(8080).setWorkerVersion(""),
DSuppliers.of(workerSetupData)
@ -101,7 +100,7 @@ public class EC2AutoScalingStrategyTest
1,
"",
new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1, Lists.<String>newArrayList(), "foo"),
new GalaxyUserData("env", "version", "type")
new GalaxyEC2UserData(new DefaultObjectMapper(), "env", "version", "type")
)
);

View File

@ -20,11 +20,17 @@
package io.druid.indexing.worker;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair;
import com.fasterxml.jackson.databind.introspect.GuiceAnnotationIntrospector;
import com.fasterxml.jackson.databind.introspect.GuiceInjectableValues;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.io.Files;
import com.google.inject.Binder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import io.druid.curator.PotentiallyGzippedCompressionProvider;
import io.druid.indexing.common.IndexingServiceCondition;
import io.druid.indexing.common.SegmentLoaderFactory;
@ -38,6 +44,7 @@ import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig;
import io.druid.indexing.overlord.ThreadPoolTaskRunner;
import io.druid.indexing.worker.config.WorkerConfig;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.LocalDataSegmentPuller;
import io.druid.segment.loading.OmniSegmentLoader;
@ -61,6 +68,40 @@ import java.util.List;
public class WorkerTaskMonitorTest
{
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
private static final Injector injector = Guice.createInjector(
new com.google.inject.Module()
{
@Override
public void configure(Binder binder)
{
binder.bind(ColumnConfig.class).toInstance(
new ColumnConfig()
{
@Override
public int columnCacheSizeBytes()
{
return 1024 * 1024;
}
}
);
}
}
);
static {
final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector();
jsonMapper.setInjectableValues(new GuiceInjectableValues(injector));
jsonMapper.setAnnotationIntrospectors(
new AnnotationIntrospectorPair(
guiceIntrospector, jsonMapper.getSerializationConfig().getAnnotationIntrospector()
),
new AnnotationIntrospectorPair(
guiceIntrospector, jsonMapper.getDeserializationConfig().getAnnotationIntrospector()
)
);
}
private static final Joiner joiner = Joiner.on("/");
private static final String basePath = "/test/druid";
private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath);
@ -121,7 +162,7 @@ public class WorkerTaskMonitorTest
workerCuratorCoordinator,
new ThreadPoolTaskRunner(
new TaskToolboxFactory(
new TaskConfig(tmp.toString(), null, null, 0),
new TaskConfig(tmp.toString(), null, null, 0, null),
null, null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory(
new OmniSegmentLoader(
ImmutableMap.<String, DataSegmentPuller>of(

View File

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

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import com.metamx.common.exception.FormattedException;
import com.metamx.common.logger.Logger;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.Firehose;
@ -115,7 +114,7 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory<ByteBufferInpu
}
@Override
public InputRow nextRow() throws FormattedException
public InputRow nextRow()
{
final byte[] message = iter.next().message();
@ -123,15 +122,7 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory<ByteBufferInpu
return null;
}
try {
return theParser.parse(ByteBuffer.wrap(message));
}
catch (Exception e) {
throw new FormattedException.Builder()
.withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW)
.withMessage(String.format("Error parsing[%s], got [%s]", ByteBuffer.wrap(message), e.toString()))
.build();
}
return theParser.parse(ByteBuffer.wrap(message));
}
@Override

View File

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

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import com.metamx.common.exception.FormattedException;
import com.metamx.common.logger.Logger;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.Firehose;
@ -123,7 +122,7 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory<ByteBufferInpu
}
@Override
public InputRow nextRow() throws FormattedException
public InputRow nextRow()
{
final Message message = iter.next().message();
@ -134,17 +133,9 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory<ByteBufferInpu
return parseMessage(message);
}
public InputRow parseMessage(Message message) throws FormattedException
public InputRow parseMessage(Message message)
{
try {
return theParser.parse(message.payload());
}
catch (Exception e) {
throw new FormattedException.Builder()
.withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW)
.withMessage(String.format("Error parsing[%s], got [%s]", message.payload(), e.toString()))
.build();
}
return theParser.parse(message.payload());
}
@Override

29
pom.xml
View File

@ -23,14 +23,14 @@
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<packaging>pom</packaging>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
<name>druid</name>
<description>druid</description>
<scm>
<connection>scm:git:ssh://git@github.com/metamx/druid.git</connection>
<developerConnection>scm:git:ssh://git@github.com/metamx/druid.git</developerConnection>
<url>http://www.github.com/metamx/druid</url>
<tag>druid-0.6.107-SNAPSHOT</tag>
<tag>druid-0.6.117-SNAPSHOT</tag>
</scm>
<prerequisites>
@ -39,9 +39,9 @@
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.25.6</metamx.java-util.version>
<apache.curator.version>2.4.0</apache.curator.version>
<druid.api.version>0.2.3</druid.api.version>
<metamx.java-util.version>0.26.5</metamx.java-util.version>
<apache.curator.version>2.5.0</apache.curator.version>
<druid.api.version>0.2.4</druid.api.version>
</properties>
<modules>
@ -59,6 +59,7 @@
<module>kafka-seven</module>
<module>kafka-eight</module>
<module>rabbitmq</module>
<module>histogram</module>
</modules>
<dependencyManagement>
@ -198,22 +199,22 @@
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<version>14.0.1</version>
<version>17.0</version>
</dependency>
<dependency>
<groupId>com.google.inject</groupId>
<artifactId>guice</artifactId>
<version>4.0-beta4</version>
<version>4.0-beta</version>
</dependency>
<dependency>
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-servlet</artifactId>
<version>4.0-beta4</version>
<version>4.0-beta</version>
</dependency>
<dependency>
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-multibindings</artifactId>
<version>4.0-beta4</version>
<version>4.0-beta</version>
</dependency>
<dependency>
<groupId>com.ibm.icu</groupId>
@ -561,15 +562,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-release-plugin</artifactId>
<version>2.4.2</version>
<dependencies>
<dependency>
<groupId>org.apache.maven.scm</groupId>
<artifactId>maven-scm-provider-gitexe</artifactId>
<!-- This version is necessary for use with git version 1.8.5 and above -->
<version>1.8.1</version>
</dependency>
</dependencies>
<version>2.5</version>
</plugin>
</plugins>
</pluginManagement>

View File

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

View File

@ -28,10 +28,8 @@ import com.google.protobuf.ByteString;
import com.google.protobuf.Descriptors;
import com.google.protobuf.DynamicMessage;
import com.google.protobuf.InvalidProtocolBufferException;
import com.metamx.common.exception.FormattedException;
import com.metamx.common.logger.Logger;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.JSONParseSpec;
import io.druid.data.input.impl.MapInputRowParser;
import io.druid.data.input.impl.ParseSpec;
@ -94,7 +92,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser
}
@Override
public InputRow parse(ByteBuffer input) throws FormattedException
public InputRow parse(ByteBuffer input)
{
// We should really create a ProtoBufBasedInputRow that does not need an intermediate map but accesses
// the DynamicMessage directly...

View File

@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.initialization;
package io.druid.guice;
import com.google.inject.Binder;
import com.google.inject.Module;

View File

@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.initialization;
package io.druid.guice;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;

View File

@ -0,0 +1,86 @@
/*
* 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.guice;
import com.google.common.collect.Lists;
import com.google.inject.Binder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.Module;
import com.google.inject.util.Modules;
import io.druid.guice.ConfigModule;
import io.druid.guice.DruidGuiceExtensions;
import io.druid.guice.DruidSecondaryModule;
import io.druid.guice.ExtensionsConfig;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.PropertiesModule;
import io.druid.jackson.JacksonModule;
import java.util.List;
/**
*/
public class GuiceInjectors
{
public static Injector makeStartupInjector()
{
return Guice.createInjector(
new DruidGuiceExtensions(),
new JacksonModule(),
new PropertiesModule("runtime.properties"),
new ConfigModule(),
new Module()
{
@Override
public void configure(Binder binder)
{
binder.bind(DruidSecondaryModule.class);
JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class);
}
}
);
}
public static Injector makeStartupInjectorWithModules(Iterable<Module> modules)
{
List<Module> theModules = Lists.newArrayList();
theModules.add(new DruidGuiceExtensions());
theModules.add(new JacksonModule());
theModules.add(new PropertiesModule("runtime.properties"));
theModules.add(new ConfigModule());
theModules.add(
new Module()
{
@Override
public void configure(Binder binder)
{
binder.bind(DruidSecondaryModule.class);
JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class);
}
}
);
for (Module theModule : modules) {
theModules.add(theModule);
}
return Guice.createInjector(theModules);
}
}

View File

@ -0,0 +1,94 @@
/*
* 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.guice;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
import com.metamx.common.ISE;
import io.druid.guice.annotations.Json;
import io.druid.guice.annotations.Smile;
import io.druid.initialization.DruidModule;
import java.util.Collections;
import java.util.List;
/**
*/
public class ModuleList
{
private final Injector baseInjector;
private final ObjectMapper jsonMapper;
private final ObjectMapper smileMapper;
private final List<Module> modules;
public ModuleList(Injector baseInjector)
{
this.baseInjector = baseInjector;
this.jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class));
this.smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class));
this.modules = Lists.newArrayList();
}
public List<Module> getModules()
{
return Collections.unmodifiableList(modules);
}
public void addModule(Object input)
{
if (input instanceof DruidModule) {
baseInjector.injectMembers(input);
modules.add(registerJacksonModules(((DruidModule) input)));
} else if (input instanceof Module) {
baseInjector.injectMembers(input);
modules.add((Module) input);
} else if (input instanceof Class) {
if (DruidModule.class.isAssignableFrom((Class) input)) {
modules.add(registerJacksonModules(baseInjector.getInstance((Class<? extends DruidModule>) input)));
} else if (Module.class.isAssignableFrom((Class) input)) {
modules.add(baseInjector.getInstance((Class<? extends Module>) input));
return;
} else {
throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class);
}
} else {
throw new ISE("Unknown module type[%s]", input.getClass());
}
}
public void addModules(Object... object)
{
for (Object o : object) {
addModule(o);
}
}
private DruidModule registerJacksonModules(DruidModule module)
{
for (com.fasterxml.jackson.databind.Module jacksonModule : module.getJacksonModules()) {
jsonMapper.registerModule(jacksonModule);
smileMapper.registerModule(jacksonModule);
}
return module;
}
}

View File

@ -17,13 +17,13 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.initialization;
package io.druid.guice;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.google.common.io.Closeables;
import com.google.inject.Binder;
import com.google.inject.Module;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.logger.Logger;
import java.io.BufferedInputStream;
@ -80,7 +80,7 @@ public class PropertiesModule implements Module
log.wtf(e, "This can only happen if the .exists() call lied. That's f'd up.");
}
finally {
Closeables.closeQuietly(stream);
CloseQuietly.close(stream);
}
binder.bind(Properties.class).toInstance(props);

View File

@ -17,14 +17,13 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server;
package io.druid.query;
import com.metamx.common.concurrent.ExecutorServiceConfig;
import io.druid.segment.column.ColumnConfig;
import org.skife.config.Config;
/**
*/
public abstract class DruidProcessingConfig extends ExecutorServiceConfig
public abstract class DruidProcessingConfig extends ExecutorServiceConfig implements ColumnConfig
{
@Config({"druid.computation.buffer.size", "${base_path}.buffer.sizeBytes"})
public int intermediateComputeSizeBytes()
@ -39,4 +38,10 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig
final int processors = Runtime.getRuntime().availableProcessors();
return processors > 1 ? processors - 1 : processors;
}
@Config(value = "${base_path}.columnCache.sizeBytes")
public int columnCacheSizeBytes()
{
return 1024 * 1024;
}
}

View File

@ -692,12 +692,14 @@ public class Druids
{
private DataSource dataSource;
private QuerySegmentSpec querySegmentSpec;
private String bound;
private Map<String, Object> context;
public TimeBoundaryQueryBuilder()
{
dataSource = null;
querySegmentSpec = null;
bound = null;
context = null;
}
@ -706,6 +708,7 @@ public class Druids
return new TimeBoundaryQuery(
dataSource,
querySegmentSpec,
bound,
context
);
}
@ -715,6 +718,7 @@ public class Druids
return new TimeBoundaryQueryBuilder()
.dataSource(builder.dataSource)
.intervals(builder.querySegmentSpec)
.bound(builder.bound)
.context(builder.context);
}
@ -748,6 +752,12 @@ public class Druids
return this;
}
public TimeBoundaryQueryBuilder bound(String b)
{
bound = b;
return this;
}
public TimeBoundaryQueryBuilder context(Map<String, Object> c)
{
context = c;

View File

@ -19,7 +19,7 @@
package io.druid.query;
import com.google.common.io.Closeables;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.guava.ResourceClosingSequence;
import com.metamx.common.guava.Sequence;
import io.druid.segment.ReferenceCountingSegment;
@ -52,7 +52,7 @@ public class ReferenceCountingSegmentQueryRunner<T> implements QueryRunner<T>
return new ResourceClosingSequence<T>(baseSequence, closeable);
}
catch (RuntimeException e) {
Closeables.closeQuietly(closeable);
CloseQuietly.close(closeable);
throw e;
}
}

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