mirror of https://github.com/apache/druid.git
merge changes from druid-0.7.x
This commit is contained in:
commit
637bd35785
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
@ -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> }
|
||||
```
|
||||
```
|
|
@ -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>, ... ] }
|
||||
```
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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`.
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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|
|
||||
|
|
|
@ -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"|
|
||||
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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, let’s calculate a simple percentage using post aggregators. Let’s imagine our data set has a metric called "total".
|
||||
|
||||
|
@ -122,5 +121,4 @@ The format of the query JSON is as follows:
|
|||
}
|
||||
...
|
||||
}
|
||||
|
||||
```
|
||||
```
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
```
|
||||
|
|
|
@ -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
|
||||
-----------------------
|
||||
|
||||
|
|
|
@ -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).
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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")));
|
||||
}
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
@ -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>
|
File diff suppressed because it is too large
Load Diff
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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() +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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()) +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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() +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -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 + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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 + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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 + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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()) +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1 @@
|
|||
io.druid.query.aggregation.histogram.ApproximateHistogramDruidModule
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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};
|
||||
}
|
||||
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -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
|
||||
);
|
||||
|
||||
|
||||
}
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -45,7 +45,6 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void shouldMakeHDFSCompliantSegmentOutputPath()
|
||||
{
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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..");
|
||||
}
|
||||
}
|
||||
|
|
@ -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())) {
|
||||
|
|
|
@ -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>()
|
||||
{
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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
|
|
@ -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 + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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")
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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
29
pom.xml
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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...
|
||||
|
|
|
@ -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;
|
|
@ -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;
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue