diff --git a/DruidCorporateCLA.pdf b/DruidCorporateCLA.pdf index 66788a0e382..d3f24b3d79f 100644 Binary files a/DruidCorporateCLA.pdf and b/DruidCorporateCLA.pdf differ diff --git a/DruidIndividualCLA.pdf b/DruidIndividualCLA.pdf index 78648d0d9be..eee4c72eb89 100644 Binary files a/DruidIndividualCLA.pdf and b/DruidIndividualCLA.pdf differ diff --git a/build.sh b/build.sh index 25b027b5626..cc8e409cfa6 100755 --- a/build.sh +++ b/build.sh @@ -30,4 +30,4 @@ echo "For examples, see: " echo " " ls -1 examples/*/*sh echo " " -echo "See also http://druid.io/docs/0.6.81" +echo "See also http://druid.io/docs/latest" diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index e3fc69954a9..4da6b0bcd4d 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index c72acf6922a..d219b91a5b9 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/docs/content/Batch-ingestion.md b/docs/content/Batch-ingestion.md index f976841d468..3d718edde3d 100644 --- a/docs/content/Batch-ingestion.md +++ b/docs/content/Batch-ingestion.md @@ -167,9 +167,20 @@ For example, data for a day may be split by the dimension "last\_name" into two In hashed partition type, the number of partitions is determined based on the targetPartitionSize and cardinality of input set and the data is partitioned based on the hashcode of the row. It is recommended to use Hashed partition as it is more efficient than singleDimension since it does not need to determine the dimension for creating partitions. -Hashing also gives better distribution of data resulting in equal sized partitons and improving query performance +Hashing also gives better distribution of data resulting in equal sized partitions and improving query performance -To use this option, the indexer must be given a target partition size. It can then find a good set of partition ranges on its own. +To use this druid to automatically determine optimal partitions indexer must be given a target partition size. It can then find a good set of partition ranges on its own. + +#### Configuration for disabling auto-sharding and creating Fixed number of partitions + Druid can be configured to NOT run determine partitions and create a fixed number of shards by specifying numShards in hashed partitionsSpec. + e.g This configuration will skip determining optimal partitions and always create 4 shards for every segment granular interval + +```json + "partitionsSpec": { + "type": "hashed" + "numShards": 4 + } +``` |property|description|required?| |--------|-----------|---------| @@ -177,6 +188,7 @@ To use this option, the indexer must be given a target partition size. It can th |targetPartitionSize|target number of rows to include in a partition, should be a number that targets segments of 700MB\~1GB.|yes| |partitionDimension|the dimension to partition on. Leave blank to select a dimension automatically.|no| |assumeGrouped|assume input data has already been grouped on time and dimensions. This is faster, but can choose suboptimal partitions if the assumption is violated.|no| +|numShards|provides a way to manually override druid-auto sharding and specify the number of shards to create for each segment granular interval.It is only supported by hashed partitionSpec and targetPartitionSize must be set to -1|no| ### Updater job spec diff --git a/docs/content/Broker-Config.md b/docs/content/Broker-Config.md index 552320ac583..44ece979580 100644 --- a/docs/content/Broker-Config.md +++ b/docs/content/Broker-Config.md @@ -81,7 +81,7 @@ druid.server.http.numThreads=50 druid.request.logging.type=emitter druid.request.logging.feed=druid_requests -druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor", "io.druid.client.cache.CacheMonitor"] +druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"] # Emit metrics over http druid.emitter=http @@ -106,16 +106,16 @@ The broker module uses several of the default modules in [Configuration](Configu |Property|Description|Default| |--------|-----------|-------| -|`druid.broker.cache.sizeInBytes`|Maximum size of the cache. If this is zero, cache is disabled.|10485760 (10MB)| -|`druid.broker.cache.initialSize`|The initial size of the cache in bytes.|500000| -|`druid.broker.cache.logEvictionCount`|If this is non-zero, there will be an eviction of entries.|0| +|`druid.broker.cache.sizeInBytes`|Maximum cache size in bytes. Zero disables caching.|0| +|`druid.broker.cache.initialSize`|Initial size of the hashtable backing the cache.|500000| +|`druid.broker.cache.logEvictionCount`|If non-zero, log cache eviction every `logEvictionCount` items.|0| #### Memcache |Property|Description|Default| |--------|-----------|-------| -|`druid.broker.cache.expiration`|Memcache [expiration time ](https://code.google.com/p/memcached/wiki/NewCommands#Standard_Protocol).|2592000 (30 days)| -|`druid.broker.cache.timeout`|Maximum time in milliseconds to wait for a response from Memcache.|500| -|`druid.broker.cache.hosts`|Memcache hosts.|none| -|`druid.broker.cache.maxObjectSize`|Maximum object size in bytes for a Memcache object.|52428800 (50 MB)| -|`druid.broker.cache.memcachedPrefix`|Key prefix for all keys in Memcache.|druid| +|`druid.broker.cache.expiration`|Memcached [expiration time](https://code.google.com/p/memcached/wiki/NewCommands#Standard_Protocol).|2592000 (30 days)| +|`druid.broker.cache.timeout`|Maximum time in milliseconds to wait for a response from Memcached.|500| +|`druid.broker.cache.hosts`|Command separated list of Memcached hosts ``.|none| +|`druid.broker.cache.maxObjectSize`|Maximum object size in bytes for a Memcached object.|52428800 (50 MB)| +|`druid.broker.cache.memcachedPrefix`|Key prefix for all keys in Memcached.|druid| diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index d3140c45b90..35269241c72 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -4,7 +4,7 @@ layout: doc_page # Configuring Druid -This describes the basic server configuration that is loaded by all the server processes; the same file is loaded by all. See also the json "specFile" descriptions in [Realtime](Realtime.html) and [Batch-ingestion](Batch-ingestion.html). +This describes the basic server configuration that is loaded by all Druid server processes; the same file is loaded by all. See also the JSON "specFile" descriptions in [Realtime](Realtime.html) and [Batch-ingestion](Batch-ingestion.html). ## JVM Configuration Best Practices @@ -26,7 +26,7 @@ Note: as a future item, we’d like to consolidate all of the various configurat ### Emitter Module -The Druid servers emit various metrics and alerts via something we call an Emitter. There are two emitter implementations included with the code, one that just logs to log4j and one that does POSTs of JSON events to a server. The properties for using the logging emitter are described below. +The Druid servers emit various metrics and alerts via something we call an Emitter. There are two emitter implementations included with the code, one that just logs to log4j ("logging", which is used by default if no emitter is specified) and one that does POSTs of JSON events to a server ("http"). The properties for using the logging emitter are described below. |Property|Description|Default| |--------|-----------|-------| @@ -47,7 +47,7 @@ The Druid servers emit various metrics and alerts via something we call an Emitt |`druid.emitter.http.timeOut`|The timeout for data reads.|PT5M| |`druid.emitter.http.flushMillis`|How often to internal message buffer is flushed (data is sent).|60000| |`druid.emitter.http.flushCount`|How many messages can the internal message buffer hold before flushing (sending).|500| -|`druid.emitter.http.recipientBaseUrl`|The base URL to emit messages to.|none| +|`druid.emitter.http.recipientBaseUrl`|The base URL to emit messages to. Druid will POST JSON to be consumed at the HTTP endpoint specified by this property.|none| ### Http Client Module @@ -56,7 +56,7 @@ This is the HTTP client used by [Broker](Broker.html) nodes. |Property|Description|Default| |--------|-----------|-------| |`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to historical and real-time nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|5| -|`druid.broker.http.readTimeout`|The timeout for data reads.|none| +|`druid.broker.http.readTimeout`|The timeout for data reads.|PT15M| ### Curator Module @@ -64,17 +64,17 @@ Druid uses [Curator](http://curator.incubator.apache.org/) for all [Zookeeper](h |Property|Description|Default| |--------|-----------|-------| -|`druid.zk.service.host`|The Zookeeper hosts to connect to.|none| -|`druid.zk.service.sessionTimeoutMs`|Zookeeper session timeout.|30000| +|`druid.zk.service.host`|The ZooKeeper hosts to connect to. This is a REQUIRED property and therefore a host address must be supplied.|none| +|`druid.zk.service.sessionTimeoutMs`|ZooKeeper session timeout, in milliseconds.|30000| |`druid.curator.compress`|Boolean flag for whether or not created Znodes should be compressed.|false| ### Announcer Module -The announcer module is used to announce and unannounce Znodes in Zookeeper (using Curator). +The announcer module is used to announce and unannounce Znodes in ZooKeeper (using Curator). -#### Zookeeper Paths +#### ZooKeeper Paths -See [Zookeeper](Zookeeper.html). +See [ZooKeeper](ZooKeeper.html). #### Data Segment Announcer @@ -84,11 +84,11 @@ Data segment announcers are used to announce segments. |--------|-----------|-------| |`druid.announcer.type`|Choices: legacy or batch. The type of data segment announcer to use.|legacy| -#### Single Data Segment Announcer +##### Single Data Segment Announcer In legacy Druid, each segment served by a node would be announced as an individual Znode. -#### Batch Data Segment Announcer +##### Batch Data Segment Announcer In current Druid, multiple data segments may be announced under the same Znode. @@ -105,16 +105,8 @@ This module contains query processing functionality. |--------|-----------|-------| |`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| |`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s| -|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, this means that even under heavy load there will still be one core available to do background tasks like talking with ZK and pulling down segments.|1| +|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| -### AWS Module - -This module is used to interact with S3. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.s3.accessKey`|The access key to use to access S3.|none| -|`druid.s3.secretKey`|The secret key to use to access S3.|none| ### Metrics Module @@ -123,7 +115,15 @@ The metrics module is used to track Druid metrics. |Property|Description|Default| |--------|-----------|-------| |`druid.monitoring.emissionPeriod`|How often metrics are emitted.|PT1m| -|`druid.monitoring.monitors`|List of Druid monitors.|none| +|`druid.monitoring.monitors`|Sets list of Druid monitors used by a node. Each monitor is specified as `com.metamx.metrics.` (see below for names and more information). For example, you can specify monitors for a Broker with `druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]`.|none (no monitors)| + +The following monitors are available: + +* CacheMonitor – Emits metrics (to logs) about the segment results cache for Historical and Broker nodes. Reports typical cache statistics include hits, misses, rates, and size (bytes and number of entries), as well as timeouts and and errors. +* SysMonitor – This uses the [SIGAR library](http://www.hyperic.com/products/sigar) to report on various system activities and statuses. +* ServerMonitor – Reports statistics on Historical nodes. +* JvmMonitor – Reports JVM-related statistics. +* RealtimeMetricsMonitor – Reports statistics on Realtime nodes. ### Server Module @@ -137,22 +137,24 @@ This module is used for Druid server nodes. ### Storage Node Module -This module is used by nodes that store data (historical and real-time nodes). +This module is used by nodes that store data (Historical and Realtime). |Property|Description|Default| |--------|-----------|-------| -|`druid.server.maxSize`|The maximum number of bytes worth of segments that the node wants assigned to it. This is not a limit that the historical nodes actually enforce, they just publish it to the coordinator and trust the coordinator to do the right thing|0| -|`druid.server.tier`|Druid server host port.|none| +|`druid.server.maxSize`|The maximum number of bytes-worth of segments that the node wants assigned to it. This is not a limit that Historical nodes actually enforce, just a value published to the Coordinator node so it can plan accordingly.|0| +|`druid.server.tier`| A string to name the distribution tier that the storage node belongs to. Many of the [rules Coordinator nodes use](Rule-Configuration.html) to manage segments can be keyed on tiers. | `_default_tier` | +|`druid.server.priority`|In a tiered architecture, the priority of the tier, thus allowing control over which nodes are queried. Higher numbers mean higher priority. The default (no priority) works for architecture with no cross replication (tiers that have no data-storage overlap). Data centers typically have equal priority. | 0 | + #### Segment Cache -Druid storage nodes maintain information about segments they have already downloaded. +Druid storage nodes maintain information about segments they have already downloaded, and a disk cache to store that data. |Property|Description|Default| |--------|-----------|-------| -|`druid.segmentCache.locations`|Segments assigned to a historical node are first stored on the local file system and then served by the historical node. These locations define where that local cache resides|none| +|`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.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| +|`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 @@ -193,7 +195,7 @@ This module is required by nodes that can serve queries. |Property|Description|Default| |--------|-----------|-------| -|`druid.query.chunkPeriod`|Long interval queries may be broken into shorter interval queries.|0| +|`druid.query.chunkPeriod`|Long-interval queries (of any type) may be broken into shorter interval queries, reducing the impact on resources. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. |0 (off)| #### GroupBy Query Config @@ -210,17 +212,28 @@ This module is required by nodes that can serve queries. |--------|-----------|-------| |`druid.query.search.maxSearchLimit`|Maximum number of search results to return.|1000| + ### Discovery Module The discovery module is used for service discovery. |Property|Description|Default| |--------|-----------|-------| -|`druid.discovery.curator.path`|Services announce themselves under this Zookeeper path.|/druid/discovery| +|`druid.discovery.curator.path`|Services announce themselves under this ZooKeeper path.|/druid/discovery| + + +#### Indexing Service Discovery Module + +This module is used to find the [Indexing Service](Indexing-Service.html) using Curator service discovery. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.selectors.indexing.serviceName`|The druid.service name of the indexing service Overlord node. To start the Overlord with a different name, set it with this property. |overlord| + ### Server Inventory View Module -This module is used to read announcements of segments in Zookeeper. The configs are identical to the Announcer Module. +This module is used to read announcements of segments in ZooKeeper. The configs are identical to the Announcer Module. ### Database Connector Module @@ -228,7 +241,6 @@ These properties specify the jdbc connection and other configuration around the |Property|Description|Default| |--------|-----------|-------| -|`druid.db.connector.pollDuration`|The jdbc connection URI.|none| |`druid.db.connector.user`|The username to connect with.|none| |`druid.db.connector.password`|The password to connect with.|none| |`druid.db.connector.createTables`|If Druid requires a table and it doesn't exist, create it?|true| @@ -250,13 +262,6 @@ The Jackson Config manager reads and writes config entries from the Druid config |--------|-----------|-------| |`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1m| -### Indexing Service Discovery Module - -This module is used to find the [Indexing Service](Indexing-Service.html) using Curator service discovery. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.selectors.indexing.serviceName`|The druid.service name of the indexing service Overlord node.|none| ### DataSegment Pusher/Puller Module @@ -290,6 +295,16 @@ This deep storage is used to interface with Amazon's S3. |`druid.storage.archiveBucket`|S3 bucket name for archiving when running the indexing-service *archive task*.|none| |`druid.storage.archiveBaseKey`|S3 object key prefix for archiving.|none| +#### AWS Module + +This module is used to interact with S3. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.s3.accessKey`|The access key to use to access S3.|none| +|`druid.s3.secretKey`|The secret key to use to access S3.|none| + + #### HDFS Deep Storage This deep storage is used to interface with HDFS. diff --git a/docs/content/Examples.md b/docs/content/Examples.md index beb436c78a9..22efff51b76 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.81 +git checkout druid-0.6.101 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.81-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Indexing-Service-Config.md b/docs/content/Indexing-Service-Config.md index 19646a5ac47..445f1960d2b 100644 --- a/docs/content/Indexing-Service-Config.md +++ b/docs/content/Indexing-Service-Config.md @@ -66,7 +66,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/indexer -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.81"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -115,7 +115,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/worker -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.81","io.druid.extensions:druid-kafka-seven:0.6.81"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101","io.druid.extensions:druid-kafka-seven:0.6.101"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index 410fe8e4bce..f8cf7bfdce8 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.81"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.101"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.81","io.druid.extensions:druid-kafka-seven:0.6.81"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101","io.druid.extensions:druid-kafka-seven:0.6.101"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/SelectQuery.md b/docs/content/SelectQuery.md new file mode 100644 index 00000000000..22eb0ca276f --- /dev/null +++ b/docs/content/SelectQuery.md @@ -0,0 +1,142 @@ +--- +layout: doc_page +--- +# Select Queries +Select queries return raw Druid rows and support pagination. + +```json + { + "queryType": "select", + "dataSource": "wikipedia", + "dimensions":[], + "metrics":[], + "granularity": "all", + "intervals": [ + "2013-01-01/2013-01-02" + ], + "pagingSpec":{"pagingIdentifiers": {}, "threshold":5} + } +``` + +There are several main parts to a select query: + +|property|description|required?| +|--------|-----------|---------| +|queryType|This String should always be "select"; this is the first thing Druid looks at to figure out how to interpret the query|yes| +|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| +|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| +|dimensions|The list of dimensions to select. If left empty, all dimensions are returned.|no| +|metrics|The list of metrics to select. If left empty, all metrics are returned.|no| +|pagingSpec|A JSON object indicating offsets into different scanned segments. Select query results will return a pagingSpec that can be reused for pagination.|yes| +|context|An additional JSON Object which can be used to specify certain flags.|no| + +The format of the result is: + +```json + [{ + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "pagingIdentifiers" : { + "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" : 4 + }, + "events" : [ { + "segmentId" : "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9", + "offset" : 0, + "event" : { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "1", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "11._korpus_(NOVJ)", + "language" : "sl", + "newpage" : "0", + "user" : "EmausBot", + "count" : 1.0, + "added" : 39.0, + "delta" : 39.0, + "variation" : 39.0, + "deleted" : 0.0 + } + }, { + "segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9", + "offset" : 1, + "event" : { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "0", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "112_U.S._580", + "language" : "en", + "newpage" : "1", + "user" : "MZMcBride", + "count" : 1.0, + "added" : 70.0, + "delta" : 70.0, + "variation" : 70.0, + "deleted" : 0.0 + } + }, { + "segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9", + "offset" : 2, + "event" : { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "0", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "113_U.S._243", + "language" : "en", + "newpage" : "1", + "user" : "MZMcBride", + "count" : 1.0, + "added" : 77.0, + "delta" : 77.0, + "variation" : 77.0, + "deleted" : 0.0 + } + }, { + "segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9", + "offset" : 3, + "event" : { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "0", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "113_U.S._73", + "language" : "en", + "newpage" : "1", + "user" : "MZMcBride", + "count" : 1.0, + "added" : 70.0, + "delta" : 70.0, + "variation" : 70.0, + "deleted" : 0.0 + } + }, { + "segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9", + "offset" : 4, + "event" : { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "0", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "113_U.S._756", + "language" : "en", + "newpage" : "1", + "user" : "MZMcBride", + "count" : 1.0, + "added" : 68.0, + "delta" : 68.0, + "variation" : 68.0, + "deleted" : 0.0 + } + } ] + } +} ] +``` + +The result returns a global pagingSpec that can be reused for the next select query. The offset will need to be increased by 1 on the client side. \ No newline at end of file diff --git a/docs/content/Tasks.md b/docs/content/Tasks.md index 799b934bca5..80f5d0952f6 100644 --- a/docs/content/Tasks.md +++ b/docs/content/Tasks.md @@ -51,12 +51,12 @@ The Index Task is a simpler variation of the Index Hadoop task that is designed |--------|-----------|---------| |type|The task type, this should always be "index".|yes| |id|The task ID. If this is not explicitly specified, Druid generates the task ID using the name of the task file and date-time stamp. |no| -|granularitySpec|Specifies the segment chunks that the task will process. `type` is always "uniform"; `gran` sets the granularity of the chunks ("DAY" means all segments containing timestamps in the same day, while `intervals` sets the interval that the chunks will cover.|yes| +|granularitySpec|Specifies the segment chunks that the task will process. `type` is always "uniform"; `gran` sets the granularity of the chunks ("DAY" means all segments containing timestamps in the same day), while `intervals` sets the interval that the chunks will cover.|yes| |spatialDimensions|Dimensions to build spatial indexes over. See [Geographic Queries](GeographicQueries.html).|no| -|aggregators|The metrics to aggregate in the data set. For more info, see [Aggregations](Aggregations.html)|yes| +|aggregators|The metrics to aggregate in the data set. For more info, see [Aggregations](Aggregations.html).|yes| |indexGranularity|The rollup granularity for timestamps. See [Realtime Ingestion](Realtime-ingestion.html) for more information. |no| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|no| -|firehose|The input source of data. For more info, see [Firehose](Firehose.html)|yes| +|firehose|The input source of data. For more info, see [Firehose](Firehose.html).|yes| |rowFlushBoundary|Used in determining when intermediate persist should occur to disk.|no| ### Index Hadoop Task @@ -74,14 +74,14 @@ The Hadoop Index Task is used to index larger data sets that require the paralle |--------|-----------|---------| |type|The task type, this should always be "index_hadoop".|yes| |config|A Hadoop Index Config. See [Batch Ingestion](Batch-ingestion.html)|yes| -|hadoopCoordinates|The Maven \:\:\ of Hadoop to use. The default is "org.apache.hadoop:hadoop-core:1.0.3".|no| +|hadoopCoordinates|The Maven \:\:\ of Hadoop to use. The default is "org.apache.hadoop:hadoop-client:2.3.0".|no| The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopBatchIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `updaterJobSpec`. The Indexing Service takes care of setting these fields internally. #### Using your own Hadoop distribution -Druid is compiled against Apache hadoop-core 1.0.3. However, if you happen to use a different flavor of hadoop that is API compatible with hadoop-core 1.0.3, you should only have to change the hadoopCoordinates property to point to the maven artifact used by your distribution. +Druid is compiled against Apache hadoop-client 2.3.0. However, if you happen to use a different flavor of hadoop that is API compatible with hadoop-client 2.3.0, you should only have to change the hadoopCoordinates property to point to the maven artifact used by your distribution. #### Resolving dependency conflicts running HadoopIndexTask diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 468e78c310c..fb4534d43ec 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.81-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.81 +cd druid-services-0.6.101 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index 122ce70ccc4..45723c03527 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -42,7 +42,7 @@ Metrics (things to aggregate over): Setting Up ---------- -At this point, you should already have Druid downloaded and are comfortable with running a Druid cluster locally. If you are not, see [here](Tutiroal%3A-The-Druid-Cluster.html). +At this point, you should already have Druid downloaded and are comfortable with running a Druid cluster locally. If you are not, see [here](Tutorial%3A-The-Druid-Cluster.html). Let's start from our usual starting point in the tarball directory. @@ -136,7 +136,7 @@ Indexing the Data To index the data and build a Druid segment, we are going to need to submit a task to the indexing service. This task should already exist: ``` -examples/indexing/index_task.json +examples/indexing/wikipedia_index_task.json ``` Open up the file to see the following: diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 215c2d83207..6a1d4ce756b 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.81-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.81"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.81","io.druid.extensions:druid-kafka-seven:0.6.81"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.101","io.druid.extensions:druid-kafka-seven:0.6.101"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index cd2bfb5eb9a..191a52c78c9 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.81-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.81 +cd druid-services-0.6.101 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.textile b/docs/content/Twitter-Tutorial.md similarity index 79% rename from docs/content/Twitter-Tutorial.textile rename to docs/content/Twitter-Tutorial.md index 6decc746b17..ccf0e2a4b3c 100644 --- a/docs/content/Twitter-Tutorial.textile +++ b/docs/content/Twitter-Tutorial.md @@ -1,77 +1,93 @@ --- layout: doc_page --- -Greetings! We see you've taken an interest in Druid. That's awesome! Hopefully this tutorial will help clarify some core Druid concepts. We will go through one of the Real-time "Examples":Examples.html, and issue some basic Druid queries. The data source we'll be working with is the "Twitter spritzer stream":https://dev.twitter.com/docs/streaming-apis/streams/public. If you are ready to explore Druid, brave its challenges, and maybe learn a thing or two, read on! +Greetings! We see you've taken an interest in Druid. That's awesome! Hopefully this tutorial will help clarify some core Druid concepts. We will go through one of the Real-time [Examples](Examples.html), and issue some basic Druid queries. The data source we'll be working with is the [Twitter spritzer stream](https://dev.twitter.com/docs/streaming-apis/streams/public). If you are ready to explore Druid, brave its challenges, and maybe learn a thing or two, read on! -h2. Setting Up +# Setting Up There are two ways to setup Druid: download a tarball, or build it from source. -h3. Download a Tarball +# Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.81-bin.tar.gz. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: -pre. tar -zxvf druid-services-0.X.X.tar.gz +``` +tar -zxvf druid-services-0.X.X.tar.gz +``` Not too lost so far right? That's great! If you cd into the directory: -pre. cd druid-services-0.X.X +``` +cd druid-services-0.X.X +``` You should see a bunch of files: + * run_example_server.sh * run_example_client.sh * LICENSE, config, examples, lib directories -h3. Clone and Build from Source +# Clone and Build from Source The other way to setup Druid is from source via git. To do so, run these commands: -
git clone git@github.com:metamx/druid.git
+```
+git clone git@github.com:metamx/druid.git
 cd druid
 git checkout druid-0.X.X
 ./build.sh
-
+``` You should see a bunch of files: -
DruidCorporateCLA.pdf	README			common			examples		indexer			pom.xml			server
+```
+DruidCorporateCLA.pdf	README			common			examples		indexer			pom.xml			server
 DruidIndividualCLA.pdf	build.sh		doc			group_by.body		install			publications		services
 LICENSE			client			eclipse_formatting.xml	index-common		merger			realtime
-
+``` You can find the example executables in the examples/bin directory: + * run_example_server.sh * run_example_client.sh -h2. Running Example Scripts +# Running Example Scripts -Let's start doing stuff. You can start a Druid "Realtime":Realtime.html node by issuing: -./run_example_server.sh +Let's start doing stuff. You can start a Druid [Realtime](Realtime.html) node by issuing: + +``` +./run_example_server.sh +``` Select "twitter". -You'll need to register a new application with the twitter API, which only takes a minute. Go to "https://twitter.com/oauth_clients/new":https://twitter.com/oauth_clients/new and fill out the form and submit. Don't worry, the home page and callback url can be anything. This will generate keys for the Twitter example application. Take note of the values for consumer key/secret and access token/secret. +You'll need to register a new application with the twitter API, which only takes a minute. Go to [this link](https://twitter.com/oauth_clients/new":https://twitter.com/oauth_clients/new) and fill out the form and submit. Don't worry, the home page and callback url can be anything. This will generate keys for the Twitter example application. Take note of the values for consumer key/secret and access token/secret. Enter your credentials when prompted. Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below. If you see crazy exceptions, you probably typed in your login information incorrectly. -
2013-05-17 23:04:40,934 INFO [main] org.mortbay.log - Started SelectChannelConnector@0.0.0.0:8080
+
+```
+2013-05-17 23:04:40,934 INFO [main] org.mortbay.log - Started SelectChannelConnector@0.0.0.0:8080
 2013-05-17 23:04:40,935 INFO [main] com.metamx.common.lifecycle.Lifecycle$AnnotationBasedHandler - Invoking start method[public void com.metamx.druid.http.FileRequestLogger.start()] on object[com.metamx.druid.http.FileRequestLogger@42bb0406].
 2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] twitter4j.TwitterStreamImpl - Connection established.
 2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] io.druid.examples.twitter.TwitterSpritzerFirehoseFactory - Connected_to_Twitter
 2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] twitter4j.TwitterStreamImpl - Receiving status stream.
-
+``` Periodically, you'll also see messages of the form: -
2013-05-17 23:04:59,793 INFO [chief-twitterstream] io.druid.examples.twitter.TwitterSpritzerFirehoseFactory - nextRow() has returned 1,000 InputRows
-
+ +``` +2013-05-17 23:04:59,793 INFO [chief-twitterstream] io.druid.examples.twitter.TwitterSpritzerFirehoseFactory - nextRow() has returned 1,000 InputRows +``` These messages indicate you are ingesting events. The Druid real time-node ingests events in an in-memory buffer. Periodically, these events will be persisted to disk. Persisting to disk generates a whole bunch of logs: -
2013-05-17 23:06:40,918 INFO [chief-twitterstream] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - Submitting persist runnable for dataSource[twitterstream]
+```
+2013-05-17 23:06:40,918 INFO [chief-twitterstream] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - Submitting persist runnable for dataSource[twitterstream]
 2013-05-17 23:06:40,920 INFO [twitterstream-incremental-persist] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - DataSource[twitterstream], Interval[2013-05-17T23:00:00.000Z/2013-05-18T00:00:00.000Z], persisting Hydrant[FireHydrant{index=com.metamx.druid.index.v1.IncrementalIndex@126212dd, queryable=com.metamx.druid.index.IncrementalIndexSegment@64c47498, count=0}]
 2013-05-17 23:06:40,937 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Starting persist for interval[2013-05-17T23:00:00.000Z/2013-05-17T23:07:00.000Z], rows[4,666]
 2013-05-17 23:06:41,039 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - outDir[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] completed index.drd in 11 millis.
@@ -88,16 +104,20 @@ These messages indicate you are ingesting events. The Druid real time-node inges
 2013-05-17 23:06:41,425 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexIO$DefaultIndexIOHandler - Converting v8[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] to v9[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0]
 2013-05-17 23:06:41,426 INFO [twitterstream-incremental-persist] 
 ... ETC
-
+``` The logs are about building different columns, probably not the most exciting stuff (they might as well be in Vulcan) if are you learning about Druid for the first time. Nevertheless, if you are interested in the details of our real-time architecture and why we persist indexes to disk, I suggest you read our "White Paper":http://static.druid.io/docs/druid.pdf. Okay, things are about to get real (-time). To query the real-time node you've spun up, you can issue: -
./run_example_client.sh
-Select "twitter" once again. This script issues ["GroupByQuery":GroupByQuery.html]s to the twitter data we've been ingesting. The query looks like this: +``` +./run_example_client.sh +``` -
{
+Select "twitter" once again. This script issues [GroupByQueries](GroupByQuery.html) to the twitter data we've been ingesting. The query looks like this:
+
+```json
+{
     "queryType": "groupBy",
     "dataSource": "twitterstream",
     "granularity": "all",
@@ -109,13 +129,14 @@ Select "twitter" once again. This script issues ["GroupByQuery":GroupByQuery.htm
     "filter": { "type": "selector", "dimension": "lang", "value": "en" },
     "intervals":["2012-10-01T00:00/2020-01-01T00"]
 }
-
+``` This is a **groupBy** query, which you may be familiar with from SQL. We are grouping, or aggregating, via the **dimensions** field: ["lang", "utc_offset"]. We are **filtering** via the **"lang"** dimension, to only look at english tweets. Our **aggregations** are what we are calculating: a row count, and the sum of the tweets in our data. The result looks something like this: -
[
+```json
+[
     {
         "version": "v1",
         "timestamp": "2012-10-01T00:00:00.000Z",
@@ -137,41 +158,48 @@ The result looks something like this:
         }
     },
 ...
-
+``` This data, plotted in a time series/distribution, looks something like this: -!http://metamarkets.com/wp-content/uploads/2013/06/tweets_timezone_offset.png(Timezone / Tweets Scatter Plot)! +![Tweets](http://metamarkets.com/wp-content/uploads/2013/06/tweets_timezone_offset.png) -This groupBy query is a bit complicated and we'll return to it later. For the time being, just make sure you are getting some blocks of data back. If you are having problems, make sure you have "curl":http://curl.haxx.se/ installed. Control+C to break out of the client script. +This groupBy query is a bit complicated and we'll return to it later. For the time being, just make sure you are getting some blocks of data back. If you are having problems, make sure you have [curl](http://curl.haxx.se/) installed. Control+C to break out of the client script. -h2. Querying Druid +# Querying Druid In your favorite editor, create the file: -
time_boundary_query.body
+ +``` +time_boundary_query.body +``` Druid queries are JSON blobs which are relatively painless to create programmatically, but an absolute pain to write by hand. So anyway, we are going to create a Druid query by hand. Add the following to the file you just created: -
{

+
+```json
+{

   "queryType"  : "timeBoundary",
   "dataSource" : "twitterstream"
 }
-
+``` The "TimeBoundaryQuery":TimeBoundaryQuery.html is one of the simplest Druid queries. To run the query, you can issue: -
 
+
+```
 curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json'  -d @time_boundary_query.body
-
+``` We get something like this JSON back: -
[ {
+```json
+{
   "timestamp" : "2013-06-10T19:09:00.000Z",
   "result" : {
     "minTime" : "2013-06-10T19:09:00.000Z",
     "maxTime" : "2013-06-10T20:50:00.000Z"
   }
 } ]
-
+``` That's the result. What information do you think the result is conveying? ... @@ -179,11 +207,14 @@ If you said the result is indicating the maximum and minimum timestamps we've se Return to your favorite editor and create the file: -
timeseries_query.body
+``` +timeseries_query.body +``` -We are going to make a slightly more complicated query, the "TimeseriesQuery":TimeseriesQuery.html. Copy and paste the following into the file: +We are going to make a slightly more complicated query, the [TimeseriesQuery](TimeseriesQuery.html). Copy and paste the following into the file: -
{
+```json
+{
   "queryType":"timeseries",
   "dataSource":"twitterstream",
   "intervals":["2010-01-01/2020-01-01"],
@@ -193,22 +224,26 @@ We are going to make a slightly more complicated query, the "TimeseriesQuery":Ti
       { "type": "doubleSum", "fieldName": "tweets", "name": "tweets"}
   ]
 }
-
+``` -You are probably wondering, what are these "Granularities":Granularities.html and "Aggregations":Aggregations.html things? What the query is doing is aggregating some metrics over some span of time. +You are probably wondering, what are these [Granularities](Granularities.html) and [Aggregations](Aggregations.html) things? What the query is doing is aggregating some metrics over some span of time. To issue the query and get some results, run the following in your command line: -
curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json'  -d @timeseries_query.body
+ +``` +curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json' -d @timeseries_query.body +``` Once again, you should get a JSON blob of text back with your results, that looks something like this: -
[ {
+```json
+[ {
   "timestamp" : "2013-06-10T19:09:00.000Z",
   "result" : {
     "tweets" : 358562.0,
     "rows" : 272271
   }
 } ]
-
+``` If you issue the query again, you should notice your results updating. @@ -216,7 +251,8 @@ Right now all the results you are getting back are being aggregated into a singl If you loudly exclaimed "we can change granularity to minute", you are absolutely correct again! We can specify different granularities to bucket our results, like so: -
{
+```json
+{
   "queryType":"timeseries",
   "dataSource":"twitterstream",
   "intervals":["2010-01-01/2020-01-01"],
@@ -226,11 +262,12 @@ If you loudly exclaimed "we can change granularity to minute", you are absolutel
       { "type": "doubleSum", "fieldName": "tweets", "name": "tweets"}
   ]
 }
-
+``` This gives us something like the following: -
[ {
+```json
+[ {
   "timestamp" : "2013-06-10T19:09:00.000Z",
   "result" : {
     "tweets" : 2650.0,
@@ -250,16 +287,21 @@ This gives us something like the following:
   }
 },
 ...
-
+``` -h2. Solving a Problem +# Solving a Problem One of Druid's main powers (see what we did there?) is to provide answers to problems, so let's pose a problem. What if we wanted to know what the top hash tags are, ordered by the number tweets, where the language is english, over the last few minutes you've been reading this tutorial? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the "GroupByQuery":GroupByQuery.html. It would be nice if we could group by results by dimension value and somehow sort those results... and it turns out we can! Let's create the file: -
group_by_query.body
+ +``` +group_by_query.body +``` and put the following in there: -
{
+
+```json
+{
     "queryType": "groupBy",
     "dataSource": "twitterstream",
     "granularity": "all",
@@ -271,16 +313,20 @@ and put the following in there:
     "filter": {"type": "selector", "dimension": "lang", "value": "en" },
     "intervals":["2012-10-01T00:00/2020-01-01T00"]
 }
-
+``` Woah! Our query just got a way more complicated. Now we have these "Filters":Filters.html things and this "OrderBy":OrderBy.html thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. If you issue the query: -
curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json'  -d @group_by_query.body
+ +``` +curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json' -d @group_by_query.body +``` You should hopefully see an answer to our question. For my twitter stream, it looks like this: -
[ {
+```json
+[ {
   "version" : "v1",
   "timestamp" : "2012-10-01T00:00:00.000Z",
   "event" : {
@@ -316,12 +362,12 @@ You should hopefully see an answer to our question. For my twitter stream, it lo
     "htags" : "IDidntTextYouBackBecause"
   }
 } ]
-
+``` Feel free to tweak other query parameters to answer other questions you may have about the data. -h2. Additional Information +# Additional Information -This tutorial is merely showcasing a small fraction of what Druid can do. Next, continue on to "The Druid Cluster":./Tutorial:-The-Druid-Cluster.html. +This tutorial is merely showcasing a small fraction of what Druid can do. Next, continue on to [The Druid Cluster](./Tutorial:-The-Druid-Cluster.html). -And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our "google groups page":http://www.groups.google.com/forum/#!forum/druid-development. +And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](http://www.groups.google.com/forum/#!forum/druid-development). diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 136aa730335..98395fc2e91 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -22,12 +22,6 @@ h2. Configuration * "Broker":Broker-Config.html * "Indexing Service":Indexing-Service-Config.html -h2. Operations -* "Extending Druid":./Modules.html -* "Cluster Setup":./Cluster-setup.html -* "Booting a Production Cluster":./Booting-a-production-cluster.html -* "Performance FAQ":./Performance-FAQ.html - h2. Data Ingestion * "Realtime":./Realtime-ingestion.html * "Batch":./Batch-ingestion.html @@ -36,6 +30,12 @@ h2. Data Ingestion * "Data Formats":./Data_formats.html * "Ingestion FAQ":./Ingestion-FAQ.html +h2. Operations +* "Extending Druid":./Modules.html +* "Cluster Setup":./Cluster-setup.html +* "Booting a Production Cluster":./Booting-a-production-cluster.html +* "Performance FAQ":./Performance-FAQ.html + h2. Querying * "Querying":./Querying.html ** "Filters":./Filters.html @@ -75,6 +75,7 @@ h2. Architecture h2. Experimental * "About Experimental Features":./About-Experimental-Features.html * "Geographic Queries":./GeographicQueries.html +* "Select Query":./SelectQuery.html h2. Development * "Versioning":./Versioning.html diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index 1dffb2cf8ff..c19813610d2 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.81"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index 94ec5bafd19..c1681adc6ff 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.81","io.druid.extensions:druid-kafka-seven:0.6.81","io.druid.extensions:druid-rabbitmq:0.6.81"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.101","io.druid.extensions:druid-kafka-seven:0.6.101","io.druid.extensions:druid-rabbitmq:0.6.101"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/examples/pom.xml b/examples/pom.xml index 26b2d2d8b46..48822dce1a9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT @@ -58,6 +58,11 @@ twitter4j-stream 3.0.3 + + commons-validator + commons-validator + 1.4.0 + @@ -82,14 +87,14 @@ ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + diff --git a/examples/src/main/java/io/druid/examples/web/WebJsonSupplier.java b/examples/src/main/java/io/druid/examples/web/WebJsonSupplier.java index cafb39c3214..9372f721145 100644 --- a/examples/src/main/java/io/druid/examples/web/WebJsonSupplier.java +++ b/examples/src/main/java/io/druid/examples/web/WebJsonSupplier.java @@ -19,8 +19,11 @@ package io.druid.examples.web; +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.common.base.Preconditions; import com.google.common.io.InputSupplier; import com.metamx.emitter.EmittingLogger; +import org.apache.commons.validator.routines.UrlValidator; import java.io.BufferedReader; import java.io.IOException; @@ -31,25 +34,25 @@ import java.net.URLConnection; public class WebJsonSupplier implements InputSupplier { private static final EmittingLogger log = new EmittingLogger(WebJsonSupplier.class); + private static final UrlValidator urlValidator = new UrlValidator(); - private String urlString; private URL url; public WebJsonSupplier(String urlString) { - this.urlString = urlString; + Preconditions.checkState(urlValidator.isValid(urlString)); + try { this.url = new URL(urlString); } catch (Exception e) { - log.error(e,"Malformed url"); + throw Throwables.propagate(e); } } @Override public BufferedReader getInput() throws IOException { - URL url = new URL(urlString); URLConnection connection = url.openConnection(); connection.setDoInput(true); return new BufferedReader(new InputStreamReader(url.openStream())); diff --git a/examples/src/test/java/io/druid/examples/web/WebJsonSupplierTest.java b/examples/src/test/java/io/druid/examples/web/WebJsonSupplierTest.java index ca181427c82..436304b7614 100644 --- a/examples/src/test/java/io/druid/examples/web/WebJsonSupplierTest.java +++ b/examples/src/test/java/io/druid/examples/web/WebJsonSupplierTest.java @@ -22,15 +22,14 @@ package io.druid.examples.web; import org.junit.Test; import java.io.IOException; +import java.net.MalformedURLException; public class WebJsonSupplierTest { - @Test(expected = IOException.class) + @Test(expected = IllegalStateException.class) public void checkInvalidUrl() throws Exception { - String invalidURL = "http://invalid.url."; WebJsonSupplier supplier = new WebJsonSupplier(invalidURL); - supplier.getInput(); } } diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 6e9919a3b1f..a45575c704d 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT @@ -52,7 +52,7 @@ org.apache.hadoop - hadoop-core + hadoop-client compile diff --git a/hll/pom.xml b/hll/pom.xml index 2e0c36b5ca6..e1e534ff57d 100644 --- a/hll/pom.xml +++ b/hll/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 19af0cb921c..e3bd0445bd3 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT @@ -67,7 +67,7 @@ org.apache.hadoop - hadoop-core + hadoop-client com.fasterxml.jackson.core diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DbUpdaterJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DbUpdaterJob.java index 36b67e10c05..a238d5e0254 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DbUpdaterJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DbUpdaterJob.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.metamx.common.logger.Logger; import io.druid.db.DbConnector; import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NoneShardSpec; import org.joda.time.DateTime; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.IDBI; @@ -39,13 +40,15 @@ public class DbUpdaterJob implements Jobby private final HadoopDruidIndexerConfig config; private final IDBI dbi; + private final DbConnector dbConnector; public DbUpdaterJob( HadoopDruidIndexerConfig config ) { this.config = config; - this.dbi = new DbConnector(config.getUpdaterJobSpec(), null).getDBI(); + this.dbConnector = new DbConnector(config.getUpdaterJobSpec(), null); + this.dbi = this.dbConnector.getDBI(); } @Override @@ -61,8 +64,11 @@ public class DbUpdaterJob implements Jobby { final PreparedBatch batch = handle.prepareBatch( String.format( - "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + dbConnector.isPostgreSQL() ? + "INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)" : + "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", config.getUpdaterJobSpec().getSegmentTable() ) ); @@ -75,7 +81,7 @@ public class DbUpdaterJob implements Jobby .put("created_date", new DateTime().toString()) .put("start", segment.getInterval().getStart().toString()) .put("end", segment.getInterval().getEnd().toString()) - .put("partitioned", segment.getShardSpec().getPartitionNum()) + .put("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? 0 : 1) .put("version", segment.getVersion()) .put("used", true) .put("payload", HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(segment)) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java index ae2d61a9a93..530d155460d 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java @@ -37,6 +37,7 @@ import io.druid.indexer.granularity.UniformGranularitySpec; import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; +import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -45,6 +46,7 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Partitioner; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; @@ -65,7 +67,6 @@ import java.util.Set; */ public class DetermineHashedPartitionsJob implements Jobby { - private static final int MAX_SHARDS = 128; private static final Logger log = new Logger(DetermineHashedPartitionsJob.class); private final HadoopDruidIndexerConfig config; @@ -98,8 +99,11 @@ public class DetermineHashedPartitionsJob implements Jobby groupByJob.setOutputKeyClass(NullWritable.class); groupByJob.setOutputValueClass(NullWritable.class); groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); + groupByJob.setPartitionerClass(DetermineHashedPartitionsPartitioner.class); if (!config.getSegmentGranularIntervals().isPresent()) { groupByJob.setNumReduceTasks(1); + } else { + groupByJob.setNumReduceTasks(config.getSegmentGranularIntervals().get().size()); } JobHelper.setupClasspath(config, groupByJob); @@ -124,9 +128,6 @@ public class DetermineHashedPartitionsJob implements Jobby if (!config.getSegmentGranularIntervals().isPresent()) { final Path intervalInfoPath = config.makeIntervalInfoPath(); fileSystem = intervalInfoPath.getFileSystem(groupByJob.getConfiguration()); - if (!fileSystem.exists(intervalInfoPath)) { - throw new ISE("Path[%s] didn't exist!?", intervalInfoPath); - } List intervals = config.jsonMapper.readValue( Utils.openInputStream(groupByJob, intervalInfoPath), new TypeReference>() { @@ -144,37 +145,25 @@ public class DetermineHashedPartitionsJob implements Jobby if (fileSystem == null) { fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration()); } - if (fileSystem.exists(partitionInfoPath)) { - Long cardinality = config.jsonMapper.readValue( - Utils.openInputStream(groupByJob, partitionInfoPath), new TypeReference() - { - } - ); - int numberOfShards = (int) Math.ceil((double) cardinality / config.getTargetPartitionSize()); - - if (numberOfShards > MAX_SHARDS) { - throw new ISE( - "Number of shards [%d] exceed the maximum limit of [%d], either targetPartitionSize is too low or data volume is too high", - numberOfShards, - MAX_SHARDS - ); - } - - List actualSpecs = Lists.newArrayListWithExpectedSize(numberOfShards); - if (numberOfShards == 1) { - actualSpecs.add(new HadoopyShardSpec(new NoneShardSpec(), shardCount++)); - } else { - for (int i = 0; i < numberOfShards; ++i) { - actualSpecs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, numberOfShards), shardCount++)); - log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i)); - } - } - - shardSpecs.put(bucket, actualSpecs); - - } else { - log.info("Path[%s] didn't exist!?", partitionInfoPath); + final Long cardinality = config.jsonMapper.readValue( + Utils.openInputStream(groupByJob, partitionInfoPath), new TypeReference() + { } + ); + final int numberOfShards = (int) Math.ceil((double) cardinality / config.getTargetPartitionSize()); + + List actualSpecs = Lists.newArrayListWithExpectedSize(numberOfShards); + if (numberOfShards == 1) { + actualSpecs.add(new HadoopyShardSpec(new NoneShardSpec(), shardCount++)); + } else { + for (int i = 0; i < numberOfShards; ++i) { + actualSpecs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, numberOfShards), shardCount++)); + log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i)); + } + } + + shardSpecs.put(bucket, actualSpecs); + } config.setShardSpecs(shardSpecs); log.info( @@ -319,13 +308,6 @@ public class DetermineHashedPartitionsJob implements Jobby } } - private byte[] getDataBytes(BytesWritable writable) - { - byte[] rv = new byte[writable.getLength()]; - System.arraycopy(writable.getBytes(), 0, rv, 0, writable.getLength()); - return rv; - } - @Override public void run(Context context) throws IOException, InterruptedException @@ -353,6 +335,50 @@ public class DetermineHashedPartitionsJob implements Jobby } } } + + public static class DetermineHashedPartitionsPartitioner + extends Partitioner implements Configurable + { + private Configuration config; + private boolean determineIntervals; + private Map reducerLookup; + + @Override + public int getPartition(LongWritable interval, BytesWritable text, int numPartitions) + { + + if (config.get("mapred.job.tracker").equals("local") || determineIntervals) { + return 0; + } else { + return reducerLookup.get(interval); + } + } + + @Override + public Configuration getConf() + { + return config; + } + + @Override + public void setConf(Configuration config) + { + this.config = config; + HadoopDruidIndexerConfig hadoopConfig = HadoopDruidIndexerConfigBuilder.fromConfiguration(config); + if (hadoopConfig.getSegmentGranularIntervals().isPresent()) { + determineIntervals = false; + int reducerNumber = 0; + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Interval interval : hadoopConfig.getSegmentGranularIntervals().get()) { + builder.put(new LongWritable(interval.getStartMillis()), reducerNumber++); + } + reducerLookup = builder.build(); + } else { + determineIntervals = true; + } + } + } + } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java index 890a3516189..ddcb691ef09 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java @@ -215,23 +215,20 @@ public class DeterminePartitionsJob implements Jobby if (fileSystem == null) { fileSystem = partitionInfoPath.getFileSystem(dimSelectionJob.getConfiguration()); } - if (fileSystem.exists(partitionInfoPath)) { - List specs = config.jsonMapper.readValue( - Utils.openInputStream(dimSelectionJob, partitionInfoPath), new TypeReference>() - { - } - ); - - List actualSpecs = Lists.newArrayListWithExpectedSize(specs.size()); - for (int i = 0; i < specs.size(); ++i) { - actualSpecs.add(new HadoopyShardSpec(specs.get(i), shardCount++)); - log.info("DateTime[%s], partition[%d], spec[%s]", segmentGranularity, i, actualSpecs.get(i)); - } - - shardSpecs.put(segmentGranularity.getStart(), actualSpecs); - } else { - log.info("Path[%s] didn't exist!?", partitionInfoPath); + List specs = config.jsonMapper.readValue( + Utils.openInputStream(dimSelectionJob, partitionInfoPath), new TypeReference>() + { } + ); + + List actualSpecs = Lists.newArrayListWithExpectedSize(specs.size()); + for (int i = 0; i < specs.size(); ++i) { + actualSpecs.add(new HadoopyShardSpec(specs.get(i), shardCount++)); + log.info("DateTime[%s], partition[%d], spec[%s]", segmentGranularity, i, actualSpecs.get(i)); + } + + shardSpecs.put(segmentGranularity.getStart(), actualSpecs); + } config.setShardSpecs(shardSpecs); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java index 2076292260d..311eec6248e 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java @@ -23,6 +23,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.inject.Inject; import com.metamx.common.logger.Logger; +import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; import org.joda.time.DateTime; import org.joda.time.DateTimeComparator; @@ -56,13 +57,28 @@ public class HadoopDruidDetermineConfigurationJob implements Jobby if (config.isDeterminingPartitions()) { jobs.add(config.getPartitionsSpec().getPartitionJob(config)); } else { + int shardsPerInterval = config.getPartitionsSpec().getNumShards(); Map> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance()); int shardCount = 0; for (Interval segmentGranularity : config.getSegmentGranularIntervals().get()) { DateTime bucket = segmentGranularity.getStart(); - final HadoopyShardSpec spec = new HadoopyShardSpec(new NoneShardSpec(), shardCount++); - shardSpecs.put(bucket, Lists.newArrayList(spec)); - log.info("DateTime[%s], spec[%s]", bucket, spec); + if (shardsPerInterval > 0) { + List specs = Lists.newArrayListWithCapacity(shardsPerInterval); + for (int i = 0; i < shardsPerInterval; i++) { + specs.add( + new HadoopyShardSpec( + new HashBasedNumberedShardSpec(i, shardsPerInterval), + shardCount++ + ) + ); + } + shardSpecs.put(bucket, specs); + log.info("DateTime[%s], spec[%s]", bucket, specs); + } else { + final HadoopyShardSpec spec = new HadoopyShardSpec(new NoneShardSpec(), shardCount++); + shardSpecs.put(bucket, Lists.newArrayList(spec)); + log.info("DateTime[%s], spec[%s]", bucket, spec); + } } config.setShardSpecs(shardSpecs); } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/AbstractPartitionsSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/AbstractPartitionsSpec.java index 90fab3e0435..e0d7deb4a32 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/AbstractPartitionsSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/AbstractPartitionsSpec.java @@ -20,6 +20,7 @@ package io.druid.indexer.partitions; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; public abstract class AbstractPartitionsSpec implements PartitionsSpec @@ -28,11 +29,13 @@ public abstract class AbstractPartitionsSpec implements PartitionsSpec private final long targetPartitionSize; private final long maxPartitionSize; private final boolean assumeGrouped; + private final int numShards; public AbstractPartitionsSpec( Long targetPartitionSize, Long maxPartitionSize, - Boolean assumeGrouped + Boolean assumeGrouped, + Integer numShards ) { this.targetPartitionSize = targetPartitionSize == null ? -1 : targetPartitionSize; @@ -40,6 +43,11 @@ public abstract class AbstractPartitionsSpec implements PartitionsSpec ? (long) (this.targetPartitionSize * DEFAULT_OVERSIZE_THRESHOLD) : maxPartitionSize; this.assumeGrouped = assumeGrouped == null ? false : assumeGrouped; + this.numShards = numShards == null ? -1 : numShards; + Preconditions.checkArgument( + this.targetPartitionSize == -1 || this.numShards == -1, + "targetPartitionsSize and shardCount both cannot be set" + ); } @JsonProperty @@ -65,4 +73,10 @@ public abstract class AbstractPartitionsSpec implements PartitionsSpec { return targetPartitionSize > 0; } + + @Override + public int getNumShards() + { + return numShards; + } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/HashedPartitionsSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/HashedPartitionsSpec.java index d164cef1638..ca53b959591 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/HashedPartitionsSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/HashedPartitionsSpec.java @@ -33,10 +33,11 @@ public class HashedPartitionsSpec extends AbstractPartitionsSpec public HashedPartitionsSpec( @JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize, @JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize, - @JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped + @JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped, + @JsonProperty("numShards") @Nullable Integer numShards ) { - super(targetPartitionSize, maxPartitionSize, assumeGrouped); + super(targetPartitionSize, maxPartitionSize, assumeGrouped, numShards); } @Override diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/PartitionsSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/PartitionsSpec.java index cce5de8becf..a36555f0ea8 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/PartitionsSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/PartitionsSpec.java @@ -49,4 +49,7 @@ public interface PartitionsSpec @JsonIgnore public boolean isDeterminingPartitions(); + @JsonProperty + public int getNumShards(); + } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/RandomPartitionsSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/RandomPartitionsSpec.java index 30f13f49478..777db4cc5c8 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/RandomPartitionsSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/RandomPartitionsSpec.java @@ -21,9 +21,6 @@ package io.druid.indexer.partitions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.indexer.DetermineHashedPartitionsJob; -import io.druid.indexer.HadoopDruidIndexerConfig; -import io.druid.indexer.Jobby; import javax.annotation.Nullable; @@ -35,9 +32,10 @@ public class RandomPartitionsSpec extends HashedPartitionsSpec public RandomPartitionsSpec( @JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize, @JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize, - @JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped + @JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped, + @JsonProperty("numShards") @Nullable Integer numShards ) { - super(targetPartitionSize, maxPartitionSize, assumeGrouped); + super(targetPartitionSize, maxPartitionSize, assumeGrouped, numShards); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/SingleDimensionPartitionsSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/SingleDimensionPartitionsSpec.java index 118d1355914..7964c1cbe6f 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/SingleDimensionPartitionsSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/SingleDimensionPartitionsSpec.java @@ -41,7 +41,7 @@ public class SingleDimensionPartitionsSpec extends AbstractPartitionsSpec @JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped ) { - super(targetPartitionSize, maxPartitionSize, assumeGrouped); + super(targetPartitionSize, maxPartitionSize, assumeGrouped, null); this.partitionDimension = partitionDimension; } diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index c6bb0ba719f..ba03ca7d76f 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -216,10 +216,10 @@ public class HadoopDruidIndexerConfigTest 150 ); - Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof SingleDimensionPartitionsSpec); + Assert.assertTrue("partitionsSpec", partitionsSpec instanceof SingleDimensionPartitionsSpec); Assert.assertEquals( "getPartitionDimension", - ((SingleDimensionPartitionsSpec)partitionsSpec).getPartitionDimension(), + ((SingleDimensionPartitionsSpec) partitionsSpec).getPartitionDimension(), "foo" ); } @@ -262,10 +262,10 @@ public class HadoopDruidIndexerConfigTest 150 ); - Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof SingleDimensionPartitionsSpec); + Assert.assertTrue("partitionsSpec", partitionsSpec instanceof SingleDimensionPartitionsSpec); Assert.assertEquals( "getPartitionDimension", - ((SingleDimensionPartitionsSpec)partitionsSpec).getPartitionDimension(), + ((SingleDimensionPartitionsSpec) partitionsSpec).getPartitionDimension(), "foo" ); } @@ -311,10 +311,10 @@ public class HadoopDruidIndexerConfigTest 200 ); - Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof SingleDimensionPartitionsSpec); + Assert.assertTrue("partitionsSpec", partitionsSpec instanceof SingleDimensionPartitionsSpec); Assert.assertEquals( "getPartitionDimension", - ((SingleDimensionPartitionsSpec)partitionsSpec).getPartitionDimension(), + ((SingleDimensionPartitionsSpec) partitionsSpec).getPartitionDimension(), "foo" ); } @@ -503,7 +503,8 @@ public class HadoopDruidIndexerConfigTest } @Test - public void testRandomPartitionsSpec() throws Exception{ + public void testRandomPartitionsSpec() throws Exception + { { final HadoopDruidIndexerConfig cfg; @@ -542,12 +543,13 @@ public class HadoopDruidIndexerConfigTest 150 ); - Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof RandomPartitionsSpec); + Assert.assertTrue("partitionsSpec", partitionsSpec instanceof RandomPartitionsSpec); } } @Test - public void testHashedPartitionsSpec() throws Exception{ + public void testHashedPartitionsSpec() throws Exception + { { final HadoopDruidIndexerConfig cfg; @@ -586,7 +588,57 @@ public class HadoopDruidIndexerConfigTest 150 ); - Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof HashedPartitionsSpec); + Assert.assertTrue("partitionsSpec", partitionsSpec instanceof HashedPartitionsSpec); } } + + @Test + public void testHashedPartitionsSpecShardCount() throws Exception + { + final HadoopDruidIndexerConfig cfg; + + try { + cfg = jsonReadWriteRead( + "{" + + "\"partitionsSpec\":{" + + " \"type\":\"hashed\"," + + " \"numShards\":2" + + " }" + + "}", + HadoopDruidIndexerConfig.class + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + final PartitionsSpec partitionsSpec = cfg.getPartitionsSpec(); + + Assert.assertEquals( + "isDeterminingPartitions", + partitionsSpec.isDeterminingPartitions(), + false + ); + + Assert.assertEquals( + "getTargetPartitionSize", + partitionsSpec.getTargetPartitionSize(), + -1 + ); + + Assert.assertEquals( + "getMaxPartitionSize", + partitionsSpec.getMaxPartitionSize(), + -1 + ); + + Assert.assertEquals( + "shardCount", + partitionsSpec.getNumShards(), + 2 + ); + + Assert.assertTrue("partitionsSpec", partitionsSpec instanceof HashedPartitionsSpec); + + } } diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 83c456fcbfa..100d3a21d52 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT @@ -71,7 +71,7 @@ org.apache.hadoop - hadoop-core + hadoop-client test diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java index eaff1b9b46f..f03c552b656 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java @@ -64,7 +64,7 @@ public abstract class AbstractTask implements Task this.id = Preconditions.checkNotNull(id, "id"); this.groupId = Preconditions.checkNotNull(groupId, "groupId"); this.taskResource = Preconditions.checkNotNull(taskResource, "resource"); - this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.dataSource = Preconditions.checkNotNull(dataSource.toLowerCase(), "dataSource"); } @JsonProperty diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index ca38f90e3cc..22529c62163 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -66,7 +66,8 @@ public class HadoopIndexTask extends AbstractTask extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class); } - private static String defaultHadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3"; + public static String DEFAULT_HADOOP_COORDINATES = "org.apache.hadoop:hadoop-client:2.3.0"; + @JsonIgnore private final HadoopDruidIndexerSchema schema; @JsonIgnore @@ -102,7 +103,7 @@ public class HadoopIndexTask extends AbstractTask this.schema = schema; this.hadoopDependencyCoordinates = hadoopDependencyCoordinates == null ? Arrays.asList( - hadoopCoordinates == null ? defaultHadoopCoordinates : hadoopCoordinates + hadoopCoordinates == null ? DEFAULT_HADOOP_COORDINATES : hadoopCoordinates ) : hadoopDependencyCoordinates; } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java index 429ac34f2a7..986a5696d5b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java @@ -33,6 +33,7 @@ import io.druid.db.DbTablesConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.NoneShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; import org.skife.jdbi.v2.FoldController; @@ -193,7 +194,7 @@ public class IndexerDBCoordinator .bind("created_date", new DateTime().toString()) .bind("start", segment.getInterval().getStart().toString()) .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", segment.getShardSpec().getPartitionNum()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? 0 : 1) .bind("version", segment.getVersion()) .bind("used", true) .bind("payload", jsonMapper.writeValueAsString(segment)) diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 1c1df24d020..a4b74914eab 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 96594460983..f36bd18717f 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/pom.xml b/pom.xml index 2ae2afb18ea..87bf8b6228c 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.81-SNAPSHOT + druid-0.6.100-SNAPSHOT @@ -39,7 +39,7 @@ UTF-8 - 0.25.3 + 0.25.4 2.4.0 0.1.11 @@ -74,12 +74,12 @@ com.metamx emitter - 0.2.9 + 0.2.11 com.metamx http-client - 0.8.5 + 0.9.2 com.metamx @@ -174,6 +174,12 @@ org.apache.curator curator-framework ${apache.curator.version} + + + org.jboss.netty + netty + + org.apache.curator @@ -313,17 +319,17 @@ org.eclipse.jetty jetty-server - 9.1.3.v20140225 + 9.1.4.v20140401 org.eclipse.jetty jetty-servlet - 9.1.3.v20140225 + 9.1.4.v20140401 org.eclipse.jetty jetty-servlets - 9.1.3.v20140225 + 9.1.4.v20140401 joda-time @@ -373,7 +379,7 @@ com.google.protobuf protobuf-java - 2.4.0a + 2.5.0 io.tesla.aether @@ -402,8 +408,8 @@ org.apache.hadoop - hadoop-core - 1.0.3 + hadoop-client + 2.3.0 provided diff --git a/processing/pom.xml b/processing/pom.xml index 4169ea3be38..db10bb7a321 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index 71beaa26652..32d9c3256f4 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Duration; @@ -120,6 +121,67 @@ public abstract class BaseQuery implements Query return retVal == null ? defaultValue : retVal; } + @Override + public int getContextPriority(int defaultValue) + { + if (context == null) { + return defaultValue; + } + Object val = context.get("priority"); + if (val == null) { + return defaultValue; + } + if (val instanceof String) { + return Integer.parseInt((String) val); + } else if (val instanceof Integer) { + return (int) val; + } else { + throw new ISE("Unknown type [%s]", val.getClass()); + } + } + + @Override + public boolean getContextBySegment(boolean defaultValue) + { + return parseBoolean("bySegment", defaultValue); + } + + @Override + public boolean getContextPopulateCache(boolean defaultValue) + { + return parseBoolean("populateCache", defaultValue); + } + + @Override + public boolean getContextUseCache(boolean defaultValue) + { + return parseBoolean("useCache", defaultValue); + } + + @Override + public boolean getContextFinalize(boolean defaultValue) + { + return parseBoolean("finalize", defaultValue); + } + + private boolean parseBoolean(String key, boolean defaultValue) + { + if (context == null) { + return defaultValue; + } + Object val = context.get(key); + if (val == null) { + return defaultValue; + } + if (val instanceof String) { + return Boolean.parseBoolean((String) val); + } else if (val instanceof Boolean) { + return (boolean) val; + } else { + throw new ISE("Unknown type [%s]. Cannot parse!", val.getClass()); + } + } + protected Map computeOverridenContext(Map overrides) { Map overridden = Maps.newTreeMap(); diff --git a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index d6150f63456..44094d0216a 100644 --- a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -53,7 +53,7 @@ public class BySegmentQueryRunner implements QueryRunner @SuppressWarnings("unchecked") public Sequence run(final Query query) { - if (Boolean.parseBoolean(query.getContextValue("bySegment"))) { + if (query.getContextBySegment(false)) { final Sequence baseSequence = base.run(query); return new Sequence() { diff --git a/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java b/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java index c26bfb35706..3a3544cf47a 100644 --- a/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java +++ b/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java @@ -64,10 +64,44 @@ public class BySegmentResultValueClass @Override public String toString() { - return "BySegmentTimeseriesResultValue{" + + return "BySegmentResultValue{" + "results=" + results + ", segmentId='" + segmentId + '\'' + ", interval='" + interval + '\'' + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + BySegmentResultValueClass that = (BySegmentResultValueClass) o; + + if (interval != null ? !interval.equals(that.interval) : that.interval != null) { + return false; + } + if (results != null ? !results.equals(that.results) : that.results != null) { + return false; + } + if (segmentId != null ? !segmentId.equals(that.segmentId) : that.segmentId != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = results != null ? results.hashCode() : 0; + result = 31 * result + (segmentId != null ? segmentId.hashCode() : 0); + result = 31 * result + (interval != null ? interval.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java index 8e666c30b16..13ca4dd75df 100644 --- a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java @@ -37,7 +37,7 @@ public abstract class BySegmentSkippingQueryRunner implements QueryRunner @Override public Sequence run(Query query) { - if (Boolean.parseBoolean(query.getContextValue("bySegment"))) { + if (query.getContextBySegment(false)) { return baseRunner.run(query); } diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index d3600068a23..6d7d1ea25b5 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -35,8 +35,10 @@ import com.metamx.common.logger.Logger; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Future; /** @@ -83,7 +85,7 @@ public class ChainedExecutionQueryRunner implements QueryRunner @Override public Sequence run(final Query query) { - final int priority = Integer.parseInt((String) query.getContextValue("priority", "0")); + final int priority = query.getContextPriority(0); return new BaseSequence>( new BaseSequence.IteratorMaker>() @@ -110,7 +112,18 @@ public class ChainedExecutionQueryRunner implements QueryRunner if (input == null) { throw new ISE("Input is null?! How is this possible?!"); } - return Sequences.toList(input.run(query), Lists.newArrayList()); + + Sequence result = input.run(query); + if (result == null) { + throw new ISE("Got a null result! Segments are missing!"); + } + + List retVal = Sequences.toList(result, Lists.newArrayList()); + if (retVal == null) { + throw new ISE("Got a null list of results! WTF?!"); + } + + return retVal; } catch (Exception e) { log.error(e, "Exception with one of the sequences!"); diff --git a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java index 2880332e184..60cc710635e 100644 --- a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java +++ b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java @@ -24,7 +24,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.FinalizeMetricManipulationFn; +import io.druid.query.aggregation.IdentityMetricManipulationFn; import io.druid.query.aggregation.MetricManipulationFn; import javax.annotation.Nullable; @@ -48,62 +49,55 @@ public class FinalizeResultsQueryRunner implements QueryRunner @Override public Sequence run(final Query query) { - final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment")); - final boolean shouldFinalize = Boolean.parseBoolean(query.getContextValue("finalize", "true")); + final boolean isBySegment = query.getContextBySegment(false); + final boolean shouldFinalize = query.getContextFinalize(true); + + final Query queryToRun; + final Function finalizerFn; + final MetricManipulationFn metricManipulationFn; + if (shouldFinalize) { - Function finalizerFn; - if (isBySegment) { - finalizerFn = new Function() - { - final Function baseFinalizer = toolChest.makeMetricManipulatorFn( - query, - new MetricManipulationFn() - { - @Override - public Object manipulate(AggregatorFactory factory, Object object) - { - return factory.finalizeComputation(factory.deserialize(object)); - } - } - ); + queryToRun = query.withOverriddenContext(ImmutableMap.of("finalize", false)); + metricManipulationFn = new FinalizeMetricManipulationFn(); - @Override - @SuppressWarnings("unchecked") - public T apply(@Nullable T input) - { - Result> result = (Result>) input; - BySegmentResultValueClass resultsClass = result.getValue(); - - return (T) new Result( - result.getTimestamp(), - new BySegmentResultValueClass( - Lists.transform(resultsClass.getResults(), baseFinalizer), - resultsClass.getSegmentId(), - resultsClass.getInterval() - ) - ); - } - }; - } - else { - finalizerFn = toolChest.makeMetricManipulatorFn( - query, - new MetricManipulationFn() - { - @Override - public Object manipulate(AggregatorFactory factory, Object object) - { - return factory.finalizeComputation(object); - } - } - ); - } - - return Sequences.map( - baseRunner.run(query.withOverriddenContext(ImmutableMap.of("finalize", "false"))), - finalizerFn - ); + } else { + queryToRun = query; + metricManipulationFn = new IdentityMetricManipulationFn(); } - return baseRunner.run(query); + if (isBySegment) { + finalizerFn = new Function() + { + final Function baseFinalizer = toolChest.makePostComputeManipulatorFn( + query, + metricManipulationFn + ); + + @Override + @SuppressWarnings("unchecked") + public T apply(@Nullable T input) + { + Result> result = (Result>) input; + BySegmentResultValueClass resultsClass = result.getValue(); + + return (T) new Result( + result.getTimestamp(), + new BySegmentResultValueClass( + Lists.transform(resultsClass.getResults(), baseFinalizer), + resultsClass.getSegmentId(), + resultsClass.getInterval() + ) + ); + } + }; + } else { + finalizerFn = toolChest.makePostComputeManipulatorFn(query, metricManipulationFn); + } + + + return Sequences.map( + baseRunner.run(queryToRun), + finalizerFn + ); + } } diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index 10dde9b26ea..20817a772e5 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -83,7 +83,7 @@ public class GroupByParallelQueryRunner implements QueryRunner query, configSupplier.get() ); - final int priority = Integer.parseInt((String) query.getContextValue("priority", "0")); + final int priority = query.getContextPriority(0); if (Iterables.isEmpty(queryables)) { log.warn("No queryables found."); diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index 10a84328584..9b9c9e373f9 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -74,6 +74,13 @@ public interface Query public ContextType getContextValue(String key, ContextType defaultValue); + // For backwards compatibility + @Deprecated public int getContextPriority(int defaultValue); + @Deprecated public boolean getContextBySegment(boolean defaultValue); + @Deprecated public boolean getContextPopulateCache(boolean defaultValue); + @Deprecated public boolean getContextUseCache(boolean defaultValue); + @Deprecated public boolean getContextFinalize(boolean defaultValue); + public Query withOverriddenContext(Map contextOverride); public Query withQuerySegmentSpec(QuerySegmentSpec spec); diff --git a/processing/src/main/java/io/druid/query/QueryToolChest.java b/processing/src/main/java/io/druid/query/QueryToolChest.java index 304d3e1eb14..8299ecaad0a 100644 --- a/processing/src/main/java/io/druid/query/QueryToolChest.java +++ b/processing/src/main/java/io/druid/query/QueryToolChest.java @@ -44,8 +44,16 @@ public abstract class QueryToolChest mergeSequences(Sequence> seqOfSequences); + public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); - public abstract Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); + + public abstract Function makePreComputeManipulatorFn(QueryType query, MetricManipulationFn fn); + + public Function makePostComputeManipulatorFn(QueryType query, MetricManipulationFn fn) + { + return makePreComputeManipulatorFn(query, fn); + } + public abstract TypeReference getResultTypeReference(); public CacheStrategy getCacheStrategy(QueryType query) { diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java new file mode 100644 index 00000000000..809a903cddd --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java @@ -0,0 +1,85 @@ +/* + * 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.query.aggregation; + +import com.google.common.collect.Lists; +import com.metamx.common.ISE; +import com.metamx.common.Pair; + +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; + +public class AggregatorUtil +{ + /** + * returns the list of dependent postAggregators that should be calculated in order to calculate given postAgg + * + * @param postAggregatorList List of postAggregator, there is a restriction that the list should be in an order + * such that all the dependencies of any given aggregator should occur before that aggregator. + * See AggregatorUtilTest.testOutOfOrderPruneDependentPostAgg for example. + * @param postAggName name of the postAgg on which dependency is to be calculated + */ + public static List pruneDependentPostAgg(List postAggregatorList, String postAggName) + { + LinkedList rv = Lists.newLinkedList(); + Set deps = new HashSet<>(); + deps.add(postAggName); + // Iterate backwards to find the last calculated aggregate and add dependent aggregator as we find dependencies in reverse order + for (PostAggregator agg : Lists.reverse(postAggregatorList)) { + if (deps.contains(agg.getName())) { + rv.addFirst(agg); // add to the beginning of List + deps.remove(agg.getName()); + deps.addAll(agg.getDependentFields()); + } + } + + return rv; + } + + public static Pair, List> condensedAggregators( + List aggList, + List postAggList, + String metric + ) + { + + List condensedPostAggs = AggregatorUtil.pruneDependentPostAgg( + postAggList, + metric + ); + // calculate dependent aggregators for these postAgg + Set dependencySet = new HashSet<>(); + dependencySet.add(metric); + for (PostAggregator postAggregator : condensedPostAggs) { + dependencySet.addAll(postAggregator.getDependentFields()); + } + + List condensedAggs = Lists.newArrayList(); + for (AggregatorFactory aggregatorSpec : aggList) { + if (dependencySet.contains(aggregatorSpec.getName())) { + condensedAggs.add(aggregatorSpec); + } + } + return new Pair(condensedAggs, condensedPostAggs); + } + +} diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java index ebd4e185ea3..c7f3eba75f4 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -88,6 +88,10 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory @Override public Object deserialize(Object object) { + // handle "NaN" / "Infinity" values serialized as strings in JSON + if (object instanceof String) { + return Double.parseDouble((String) object); + } return object; } diff --git a/processing/src/main/java/io/druid/query/aggregation/FinalizeMetricManipulationFn.java b/processing/src/main/java/io/druid/query/aggregation/FinalizeMetricManipulationFn.java new file mode 100644 index 00000000000..e532421e572 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/FinalizeMetricManipulationFn.java @@ -0,0 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.aggregation; + +/** + */ +public class FinalizeMetricManipulationFn implements MetricManipulationFn +{ + @Override + public Object manipulate(AggregatorFactory factory, Object object) + { + return factory.finalizeComputation(object); + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/IdentityMetricManipulationFn.java b/processing/src/main/java/io/druid/query/aggregation/IdentityMetricManipulationFn.java new file mode 100644 index 00000000000..6b99838700a --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/IdentityMetricManipulationFn.java @@ -0,0 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.aggregation; + +/** + */ +public class IdentityMetricManipulationFn implements MetricManipulationFn +{ + @Override + public Object manipulate(AggregatorFactory factory, Object object) + { + return object; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index 6de6be09ad8..85bd9597ae8 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -139,6 +139,10 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory @Override public Object deserialize(Object object) { + // handle "NaN" / "Infinity" values serialized as strings in JSON + if (object instanceof String) { + return Double.parseDouble((String) object); + } return object; } diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java index ee8217f820b..b731c4319e7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java @@ -85,6 +85,7 @@ public class MaxAggregatorFactory implements AggregatorFactory @Override public Object deserialize(Object object) { + // handle "NaN" / "Infinity" values serialized as strings in JSON if (object instanceof String) { return Double.parseDouble((String) object); } diff --git a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java index 9c3d560bacf..d3956c94b52 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java @@ -85,6 +85,7 @@ public class MinAggregatorFactory implements AggregatorFactory @Override public Object deserialize(Object object) { + // handle "NaN" / "Infinity" values serialized as strings in JSON if (object instanceof String) { return Double.parseDouble((String) object); } diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java index bd5cfd84058..dbd766c391a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java @@ -326,27 +326,26 @@ public abstract class HyperLogLogCollector implements Comparable dimValues = inputRow.getDimension(metricName); - if (dimValues == null) { + if (rawValue instanceof HyperLogLogCollector) { + return (HyperLogLogCollector) inputRow.getRaw(metricName); + } else { + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + + List dimValues = inputRow.getDimension(metricName); + if (dimValues == null) { + return collector; + } + + for (String dimensionValue : dimValues) { + collector.add(hashFn.hashBytes(dimensionValue.getBytes(Charsets.UTF_8)).asBytes()); + } return collector; } - - for (String dimensionValue : dimValues) { - collector.add(hashFn.hashBytes(dimensionValue.getBytes(Charsets.UTF_8)).asBytes()); - } - return collector; } }; } diff --git a/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java b/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java index b9cef360238..1f8740562a3 100644 --- a/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java @@ -67,4 +67,13 @@ public class JavaScriptDimFilter implements DimFilter .put(functionBytes) .array(); } + + @Override + public String toString() + { + return "JavaScriptDimFilter{" + + "dimension='" + dimension + '\'' + + ", function='" + function + '\'' + + '}'; + } } diff --git a/processing/src/main/java/io/druid/query/filter/RegexDimFilter.java b/processing/src/main/java/io/druid/query/filter/RegexDimFilter.java index 0644250f819..9a327dde3b5 100644 --- a/processing/src/main/java/io/druid/query/filter/RegexDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/RegexDimFilter.java @@ -69,4 +69,13 @@ public class RegexDimFilter implements DimFilter .put(patternBytes) .array(); } + + @Override + public String toString() + { + return "RegexDimFilter{" + + "dimension='" + dimension + '\'' + + ", pattern='" + pattern + '\'' + + '}'; + } } diff --git a/processing/src/main/java/io/druid/query/filter/SearchQueryDimFilter.java b/processing/src/main/java/io/druid/query/filter/SearchQueryDimFilter.java index 76c5ecd0148..c6d09c0a55c 100644 --- a/processing/src/main/java/io/druid/query/filter/SearchQueryDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/SearchQueryDimFilter.java @@ -64,9 +64,18 @@ public class SearchQueryDimFilter implements DimFilter final byte[] queryBytes = query.getCacheKey(); return ByteBuffer.allocate(1 + dimensionBytes.length + queryBytes.length) - .put(DimFilterCacheHelper.SEARCH_QUERY_TYPE_ID) - .put(dimensionBytes) - .put(queryBytes) - .array(); + .put(DimFilterCacheHelper.SEARCH_QUERY_TYPE_ID) + .put(dimensionBytes) + .put(queryBytes) + .array(); + } + + @Override + public String toString() + { + return "SearchQueryDimFilter{" + + "dimension='" + dimension + '\'' + + ", query=" + query + + '}'; } } diff --git a/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java b/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java index 6899d306602..2abcc9282b3 100644 --- a/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java @@ -99,4 +99,13 @@ public class SpatialDimFilter implements DimFilter result = 31 * result + (bound != null ? bound.hashCode() : 0); return result; } + + @Override + public String toString() + { + return "SpatialDimFilter{" + + "dimension='" + dimension + '\'' + + ", bound=" + bound + + '}'; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index ea58501635b..1c75f1390f1 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -44,6 +44,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; +import io.druid.query.extraction.DimExtractionFn; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.StorageAdapter; @@ -69,7 +70,7 @@ public class GroupByQueryEngine private final StupidPool intermediateResultsBufferPool; @Inject - public GroupByQueryEngine ( + public GroupByQueryEngine( Supplier config, @Global StupidPool intermediateResultsBufferPool ) @@ -80,6 +81,12 @@ public class GroupByQueryEngine public Sequence process(final GroupByQuery query, StorageAdapter storageAdapter) { + if (storageAdapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + final List intervals = query.getQuerySegmentSpec().getIntervals(); if (intervals.size() != 1) { throw new IAE("Should only have one interval, got[%s]", intervals); @@ -182,12 +189,11 @@ public class GroupByQueryEngine final DimensionSelector dimSelector = dims.get(0); final IndexedInts row = dimSelector.getRow(); - if (row.size() == 0) { + if (row == null || row.size() == 0) { ByteBuffer newKey = key.duplicate(); newKey.putInt(dimSelector.getValueCardinality()); unaggregatedBuffers = updateValues(newKey, dims.subList(1, dims.size())); - } - else { + } else { for (Integer dimValue : row) { ByteBuffer newKey = key.duplicate(); newKey.putInt(dimValue); @@ -201,8 +207,7 @@ public class GroupByQueryEngine retVal.addAll(unaggregatedBuffers); } return retVal; - } - else { + } else { key.clear(); Integer position = positions.get(key); int[] increments = positionMaintainer.getIncrements(); @@ -266,8 +271,7 @@ public class GroupByQueryEngine { if (nextVal > max) { return null; - } - else { + } else { int retVal = (int) nextVal; nextVal += increment; return retVal; @@ -398,9 +402,14 @@ public class GroupByQueryEngine ByteBuffer keyBuffer = input.getKey().duplicate(); for (int i = 0; i < dimensions.size(); ++i) { final DimensionSelector dimSelector = dimensions.get(i); + final DimExtractionFn fn = dimensionSpecs.get(i).getDimExtractionFn(); final int dimVal = keyBuffer.getInt(); if (dimSelector.getValueCardinality() != dimVal) { - theEvent.put(dimNames.get(i), dimSelector.lookupName(dimVal)); + if (fn != null) { + theEvent.put(dimNames.get(i), fn.apply(dimSelector.lookupName(dimVal))); + } else { + theEvent.put(dimNames.get(i), dimSelector.lookupName(dimVal)); + } } } @@ -428,9 +437,10 @@ public class GroupByQueryEngine throw new UnsupportedOperationException(); } - public void close() { + public void close() + { // cleanup - for(BufferAggregator agg : aggregators) { + for (BufferAggregator agg : aggregators) { agg.close(); } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 1b77f2299ba..ab70b34db05 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -173,7 +173,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest makeMetricManipulatorFn(final GroupByQuery query, final MetricManipulationFn fn) + public Function makePreComputeManipulatorFn(final GroupByQuery query, final MetricManipulationFn fn) { return new Function() { diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 0bc9f22b1dd..98eb896476d 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -155,7 +155,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest makeMetricManipulatorFn( + public Function makePreComputeManipulatorFn( SegmentMetadataQuery query, MetricManipulationFn fn ) { diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index d600b2b4499..c5b64c2d46a 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -104,47 +104,47 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory( - Sequences.map( - Sequences.simple(queryRunners), - new Function, QueryRunner>() + Sequences.map( + Sequences.simple(queryRunners), + new Function, QueryRunner>() + { + @Override + public QueryRunner apply(final QueryRunner input) + { + return new QueryRunner() { @Override - public QueryRunner apply(final QueryRunner input) + public Sequence run(final Query query) { - return new QueryRunner() - { - @Override - public Sequence run(final Query query) - { - Future> future = queryExecutor.submit( - new Callable>() - { - @Override - public Sequence call() throws Exception - { - return new ExecutorExecutingSequence( - input.run(query), - queryExecutor - ); - } - } - ); - try { - return future.get(); + Future> future = queryExecutor.submit( + new Callable>() + { + @Override + public Sequence call() throws Exception + { + return new ExecutorExecutingSequence( + input.run(query), + queryExecutor + ); + } } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - catch (ExecutionException e) { - throw Throwables.propagate(e); - } - } - }; + ); + try { + return future.get(); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } } - } - ) - ); + }; + } + } + ) + ); } @Override diff --git a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java index 098a5462d3b..c6d6ecca0bd 100644 --- a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java @@ -32,7 +32,6 @@ import java.util.Map; public class SegmentMetadataQuery extends BaseQuery { - private final ColumnIncluderator toInclude; private final boolean merge; diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 6e14ef1c1f3..a8429ad2726 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -129,7 +129,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest, Result> makeMetricManipulatorFn( + public Function, Result> makePreComputeManipulatorFn( SearchQuery query, MetricManipulationFn fn ) { @@ -294,7 +294,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest> { private static final EmittingLogger log = new EmittingLogger(SearchQueryRunner.class); @@ -99,12 +99,10 @@ public class SearchQueryRunner implements QueryRunner> ConciseSet set = new ConciseSet(); set.add(0); baseFilter = ImmutableConciseSet.newImmutableFromMutable(set); - } - else { + } else { baseFilter = ImmutableConciseSet.complement(new ImmutableConciseSet(), index.getNumRows()); } - } - else { + } else { baseFilter = filter.goConcise(new ColumnSelectorBitmapIndexSelector(index)); } @@ -133,49 +131,52 @@ public class SearchQueryRunner implements QueryRunner> } final StorageAdapter adapter = segment.asStorageAdapter(); - if (adapter != null) { - Iterable dimsToSearch; - if (dimensions == null || dimensions.isEmpty()) { - dimsToSearch = adapter.getAvailableDimensions(); - } else { - dimsToSearch = dimensions; + + if (adapter == null) { + log.makeAlert("WTF!? Unable to process search query on segment.") + .addData("segment", segment.getIdentifier()) + .addData("query", query).emit(); + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + + Iterable dimsToSearch; + if (dimensions == null || dimensions.isEmpty()) { + dimsToSearch = adapter.getAvailableDimensions(); + } else { + dimsToSearch = dimensions; + } + + final TreeSet retVal = Sets.newTreeSet(query.getSort().getComparator()); + + final Iterable cursors = adapter.makeCursors(filter, segment.getDataInterval(), QueryGranularity.ALL); + for (Cursor cursor : cursors) { + Map dimSelectors = Maps.newHashMap(); + for (String dim : dimsToSearch) { + dimSelectors.put(dim, cursor.makeDimensionSelector(dim)); } - final TreeSet retVal = Sets.newTreeSet(query.getSort().getComparator()); - - final Iterable cursors = adapter.makeCursors(filter, segment.getDataInterval(), QueryGranularity.ALL); - for (Cursor cursor : cursors) { - Map dimSelectors = Maps.newHashMap(); - for (String dim : dimsToSearch) { - dimSelectors.put(dim, cursor.makeDimensionSelector(dim)); - } - - while (!cursor.isDone()) { - for (Map.Entry entry : dimSelectors.entrySet()) { - final DimensionSelector selector = entry.getValue(); - final IndexedInts vals = selector.getRow(); - for (int i = 0; i < vals.size(); ++i) { - final String dimVal = selector.lookupName(vals.get(i)); - if (searchQuerySpec.accept(dimVal)) { - retVal.add(new SearchHit(entry.getKey(), dimVal)); - if (retVal.size() >= limit) { - return makeReturnResult(limit, retVal); - } + while (!cursor.isDone()) { + for (Map.Entry entry : dimSelectors.entrySet()) { + final DimensionSelector selector = entry.getValue(); + final IndexedInts vals = selector.getRow(); + for (int i = 0; i < vals.size(); ++i) { + final String dimVal = selector.lookupName(vals.get(i)); + if (searchQuerySpec.accept(dimVal)) { + retVal.add(new SearchHit(entry.getKey(), dimVal)); + if (retVal.size() >= limit) { + return makeReturnResult(limit, retVal); } } } - - cursor.advance(); } - } - return makeReturnResult(limit, retVal); + cursor.advance(); + } } - log.makeAlert("WTF!? Unable to process search query on segment.") - .addData("segment", segment.getIdentifier()) - .addData("query", query); - return Sequences.empty(); + return makeReturnResult(limit, retVal); } private Sequence> makeReturnResult(int limit, TreeSet retVal) diff --git a/processing/src/main/java/io/druid/query/select/EventHolder.java b/processing/src/main/java/io/druid/query/select/EventHolder.java index 1ac3661d1f5..aa4f6418289 100644 --- a/processing/src/main/java/io/druid/query/select/EventHolder.java +++ b/processing/src/main/java/io/druid/query/select/EventHolder.java @@ -22,6 +22,7 @@ package io.druid.query.select; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; +import com.metamx.common.ISE; import org.joda.time.DateTime; import java.util.Map; @@ -50,7 +51,14 @@ public class EventHolder public DateTime getTimestamp() { - return (DateTime) event.get(timestampKey); + Object retVal = event.get(timestampKey); + if (retVal instanceof String) { + return new DateTime(retVal); + } else if (retVal instanceof DateTime) { + return (DateTime) retVal; + } else { + throw new ISE("Do not understand format [%s]", retVal.getClass()); + } } @JsonProperty diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java index 3238ac01f7a..f5bc7aba043 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java @@ -22,6 +22,7 @@ package io.druid.query.select; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; import io.druid.query.QueryRunnerHelper; @@ -54,6 +55,12 @@ public class SelectQueryEngine { final StorageAdapter adapter = segment.asStorageAdapter(); + if (adapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + final Iterable dims; if (query.getDimensions() == null || query.getDimensions().isEmpty()) { dims = adapter.getAvailableDimensions(); diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 1533c48c383..f3bbe028ed8 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -131,7 +131,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest, Result> makeMetricManipulatorFn( + public Function, Result> makePreComputeManipulatorFn( final SelectQuery query, final MetricManipulationFn fn ) { @@ -170,10 +170,9 @@ public class SelectQueryQueryToolChest extends QueryToolChest metrics = Sets.newTreeSet(); if (query.getMetrics() != null) { - dimensions.addAll(query.getMetrics()); + metrics.addAll(query.getMetrics()); } final byte[][] metricBytes = new byte[metrics.size()][]; diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 95715bb7511..7186b1a6e38 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -123,7 +123,7 @@ public class TimeBoundaryQueryQueryToolChest } @Override - public Function, Result> makeMetricManipulatorFn( + public Function, Result> makePreComputeManipulatorFn( TimeBoundaryQuery query, MetricManipulationFn fn ) { diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index cd732de18e4..16e9ae832fa 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -87,6 +87,12 @@ public class TimeBoundaryQueryRunnerFactory @Override public Iterator> make() { + if (adapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + return legacyQuery.buildResult( adapter.getInterval().getStart(), adapter.getMinTime(), diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesBinaryFn.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesBinaryFn.java index f8530ce334f..7cecb554a92 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesBinaryFn.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesBinaryFn.java @@ -24,7 +24,6 @@ import io.druid.granularity.AllGranularity; import io.druid.granularity.QueryGranularity; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.PostAggregator; import java.util.LinkedHashMap; import java.util.List; @@ -37,17 +36,14 @@ public class TimeseriesBinaryFn { private final QueryGranularity gran; private final List aggregations; - private final List postAggregations; public TimeseriesBinaryFn( QueryGranularity granularity, - List aggregations, - List postAggregations + List aggregations ) { this.gran = granularity; this.aggregations = aggregations; - this.postAggregations = postAggregations; } @Override @@ -71,11 +67,6 @@ public class TimeseriesBinaryFn retVal.put(metricName, factory.combine(arg1Val.getMetric(metricName), arg2Val.getMetric(metricName))); } - for (PostAggregator pf : postAggregations) { - final String metricName = pf.getName(); - retVal.put(metricName, pf.compute(retVal)); - } - return (gran instanceof AllGranularity) ? new Result( arg1.getTimestamp(), diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java index b57105aa654..6ab42477890 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java @@ -20,6 +20,7 @@ package io.druid.query.timeseries; import com.google.common.base.Function; +import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; import io.druid.query.QueryRunnerHelper; @@ -40,6 +41,12 @@ public class TimeseriesQueryEngine { public Sequence> process(final TimeseriesQuery query, final StorageAdapter adapter) { + if (adapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + return new BaseSequence, Iterator>>( new BaseSequence.IteratorMaker, Iterator>>() { @@ -74,10 +81,6 @@ public class TimeseriesQueryEngine bob.addMetric(aggregator); } - for (PostAggregator postAgg : postAggregatorSpecs) { - bob.addMetric(postAgg); - } - Result retVal = bob.build(); // cleanup diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index b4752266944..482e92adc3e 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -101,8 +101,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest, Result> makeMetricManipulatorFn( - final TimeseriesQuery query, final MetricManipulationFn fn - ) - { - return new Function, Result>() - { - @Override - public Result apply(Result result) - { - final Map values = Maps.newHashMap(); - final TimeseriesResultValue holder = result.getValue(); - for (AggregatorFactory agg : query.getAggregatorSpecs()) { - values.put(agg.getName(), fn.manipulate(agg, holder.getMetric(agg.getName()))); - } - for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { - values.put(postAgg.getName(), holder.getMetric(postAgg.getName())); - } - return new Result( - result.getTimestamp(), - new TimeseriesResultValue(values) - ); - } - }; - } - @Override public TypeReference> getResultTypeReference() { @@ -169,7 +142,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest, Object, TimeseriesQuery>() { private final List aggs = query.getAggregatorSpecs(); - private final List postAggs = query.getPostAggregatorSpecs(); @Override public byte[] computeCacheKey(TimeseriesQuery query) @@ -238,10 +210,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest( timestamp, new TimeseriesResultValue(retVal) @@ -268,4 +236,52 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest, Result> makePreComputeManipulatorFn( + final TimeseriesQuery query, final MetricManipulationFn fn + ) + { + return makeComputeManipulatorFn(query, fn, false); + } + + @Override + public Function, Result> makePostComputeManipulatorFn( + TimeseriesQuery query, MetricManipulationFn fn + ) + { + return makeComputeManipulatorFn(query, fn, true); + } + + private Function, Result> makeComputeManipulatorFn( + final TimeseriesQuery query, final MetricManipulationFn fn, final boolean calculatePostAggs + ) + { + return new Function, Result>() + { + @Override + public Result apply(Result result) + { + final Map values = Maps.newHashMap(); + final TimeseriesResultValue holder = result.getValue(); + if (calculatePostAggs) { + // put non finalized aggregators for calculating dependent post Aggregators + for (AggregatorFactory agg : query.getAggregatorSpecs()) { + values.put(agg.getName(), holder.getMetric(agg.getName())); + } + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + values.put(postAgg.getName(), postAgg.compute(values)); + } + } + for (AggregatorFactory agg : query.getAggregatorSpecs()) { + values.put(agg.getName(), fn.manipulate(agg, holder.getMetric(agg.getName()))); + } + + return new Result( + result.getTimestamp(), + new TimeseriesResultValue(values) + ); + } + }; + } } diff --git a/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java index 84be056fea1..254d13d581b 100644 --- a/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java @@ -19,10 +19,11 @@ package io.druid.query.topn; -import com.google.common.collect.Lists; import com.metamx.common.ISE; +import com.metamx.common.Pair; import io.druid.collections.StupidPool; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.PostAggregator; import io.druid.segment.Capabilities; import io.druid.segment.Cursor; @@ -56,7 +57,6 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm, List> condensedAggPostAggPair = AggregatorUtil.condensedAggregators( + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs(), + metric + ); - // Find either the aggregator or post aggregator to do the topN over - List condensedAggs = Lists.newArrayList(); - for (AggregatorFactory aggregatorSpec : query.getAggregatorSpecs()) { - if (aggregatorSpec.getName().equalsIgnoreCase(metric)) { - condensedAggs.add(aggregatorSpec); - break; - } - } - List condensedPostAggs = Lists.newArrayList(); - if (condensedAggs.isEmpty()) { - for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { - if (postAggregator.getName().equalsIgnoreCase(metric)) { - condensedPostAggs.add(postAggregator); - - // Add all dependent metrics - for (AggregatorFactory aggregatorSpec : query.getAggregatorSpecs()) { - if (postAggregator.getDependentFields().contains(aggregatorSpec.getName())) { - condensedAggs.add(aggregatorSpec); - } - } - break; - } - } - } - if (condensedAggs.isEmpty() && condensedPostAggs.isEmpty()) { + if (condensedAggPostAggPair.lhs.isEmpty() && condensedAggPostAggPair.rhs.isEmpty()) { throw new ISE("WTF! Can't find the metric to do topN over?"); } - // Run topN for only a single metric TopNQuery singleMetricQuery = new TopNQueryBuilder().copy(query) - .aggregators(condensedAggs) - .postAggregators(condensedPostAggs) + .aggregators(condensedAggPostAggPair.lhs) + .postAggregators(condensedAggPostAggPair.rhs) .build(); + final TopNResultBuilder singleMetricResultBuilder = BaseTopNAlgorithm.makeResultBuilder(params, singleMetricQuery); PooledTopNAlgorithm singleMetricAlgo = new PooledTopNAlgorithm(capabilities, singleMetricQuery, bufferPool); PooledTopNAlgorithm.PooledTopNParams singleMetricParam = null; diff --git a/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java index 0c32d8db676..9d8baceb2c9 100644 --- a/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java @@ -28,6 +28,7 @@ import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import java.util.Arrays; +import java.util.Comparator; import java.util.List; /** @@ -230,4 +231,18 @@ public abstract class BaseTopNAlgorithm aggFactories, + List postAggs ) { - return delegate.getResultBuilder(timestamp, dimSpec, threshold, comparator); + return delegate.getResultBuilder(timestamp, dimSpec, threshold, comparator, aggFactories, postAggs); } @Override @@ -102,15 +103,27 @@ public class InvertedTopNMetricSpec implements TopNMetricSpec delegate.initTopNAlgorithmSelector(selector); } + @Override + public String getMetricName(DimensionSpec dimSpec) + { + return delegate.getMetricName(dimSpec); + } + @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } InvertedTopNMetricSpec that = (InvertedTopNMetricSpec) o; - if (delegate != null ? !delegate.equals(that.delegate) : that.delegate != null) return false; + if (delegate != null ? !delegate.equals(that.delegate) : that.delegate != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java index 0f2c8344b14..b7c7c6a2565 100644 --- a/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java +++ b/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java @@ -80,10 +80,12 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec DateTime timestamp, DimensionSpec dimSpec, int threshold, - Comparator comparator + Comparator comparator, + List aggFactories, + List postAggs ) { - return new TopNLexicographicResultBuilder(timestamp, dimSpec, threshold, previousStop, comparator); + return new TopNLexicographicResultBuilder(timestamp, dimSpec, threshold, previousStop, comparator, aggFactories); } @Override @@ -111,6 +113,12 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec selector.setAggregateAllMetrics(true); } + @Override + public String getMetricName(DimensionSpec dimSpec) + { + return dimSpec.getOutputName(); + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java index 76f1a9341ac..9ad97e239cd 100644 --- a/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java +++ b/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java @@ -121,10 +121,12 @@ public class NumericTopNMetricSpec implements TopNMetricSpec DateTime timestamp, DimensionSpec dimSpec, int threshold, - Comparator comparator + Comparator comparator, + List aggFactories, + List postAggs ) { - return new TopNNumericResultBuilder(timestamp, dimSpec, metric, threshold, comparator); + return new TopNNumericResultBuilder(timestamp, dimSpec, metric, threshold, comparator, aggFactories, postAggs); } @Override @@ -150,6 +152,12 @@ public class NumericTopNMetricSpec implements TopNMetricSpec selector.setAggregateTopNMetricFirst(true); } + @Override + public String getMetricName(DimensionSpec dimSpec) + { + return metric; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java index d87631c7b57..a8e3f324467 100644 --- a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -35,7 +35,8 @@ import java.util.Comparator; /** */ -public class PooledTopNAlgorithm extends BaseTopNAlgorithm +public class PooledTopNAlgorithm + extends BaseTopNAlgorithm { private final Capabilities capabilities; private final TopNQuery query; @@ -113,13 +114,7 @@ public class PooledTopNAlgorithm extends BaseTopNAlgorithm resultsBufHolder; private final ByteBuffer resultsBuf; private final int[] aggregatorSizes; @@ -278,6 +266,11 @@ public class PooledTopNAlgorithm extends BaseTopNAlgorithm getResultsBufHolder() { return resultsBufHolder; diff --git a/processing/src/main/java/io/druid/query/topn/TopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/TopNAlgorithm.java index 89bac871441..3d5c90e2b20 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/TopNAlgorithm.java @@ -33,8 +33,6 @@ public interface TopNAlgorithm public TopNParams makeInitParams(DimensionSelector dimSelector, Cursor cursor); - public TopNResultBuilder makeResultBuilder(Parameters params); - public void run( Parameters params, TopNResultBuilder resultBuilder, diff --git a/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java b/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java index 437c28f640f..4c02da447aa 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java +++ b/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java @@ -24,6 +24,7 @@ import io.druid.granularity.AllGranularity; import io.druid.granularity.QueryGranularity; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; import org.joda.time.DateTime; @@ -63,7 +64,11 @@ public class TopNBinaryFn implements BinaryFn, Result, Result retVals = new LinkedHashMap(); + Map retVals = new LinkedHashMap<>(); TopNResultValue arg1Vals = arg1.getValue(); TopNResultValue arg2Vals = arg2.getValue(); @@ -92,7 +97,8 @@ public class TopNBinaryFn implements BinaryFn, Result retVal = new LinkedHashMap(); + // size of map = aggregator + topNDim + postAgg (If sorting is done on post agg field) + Map retVal = new LinkedHashMap<>(aggregations.size() + 2); retVal.put(dimension, dimensionValue); for (AggregatorFactory factory : aggregations) { @@ -117,7 +123,14 @@ public class TopNBinaryFn implements BinaryFn, Result aggFactories; private MinMaxPriorityQueue pQueue = null; public TopNLexicographicResultBuilder( @@ -48,12 +48,14 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder DimensionSpec dimSpec, int threshold, String previousStop, - final Comparator comparator + final Comparator comparator, + List aggFactories ) { this.timestamp = timestamp; this.dimSpec = dimSpec; this.previousStop = previousStop; + this.aggFactories = aggFactories; instantiatePQueue(threshold, comparator); } @@ -62,9 +64,7 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder public TopNResultBuilder addEntry( String dimName, Object dimValIndex, - Object[] metricVals, - List aggFactories, - List postAggs + Object[] metricVals ) { Map metricValues = Maps.newLinkedHashMap(); @@ -75,9 +75,6 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder for (Object metricVal : metricVals) { metricValues.put(aggsIter.next().getName(), metricVal); } - for (PostAggregator postAgg : postAggs) { - metricValues.put(postAgg.getName(), postAgg.compute(metricValues)); - } pQueue.add(new DimValHolder.Builder().withDirName(dimName).withMetricValues(metricValues).build()); } diff --git a/processing/src/main/java/io/druid/query/topn/TopNMapFn.java b/processing/src/main/java/io/druid/query/topn/TopNMapFn.java index c013d546f7f..5479eeb72f8 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNMapFn.java +++ b/processing/src/main/java/io/druid/query/topn/TopNMapFn.java @@ -24,6 +24,8 @@ import io.druid.query.Result; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; +import java.util.Comparator; + public class TopNMapFn implements Function> { private final TopNQuery query; @@ -52,7 +54,7 @@ public class TopNMapFn implements Function> try { params = topNAlgorithm.makeInitParams(dimSelector, cursor); - TopNResultBuilder resultBuilder = topNAlgorithm.makeResultBuilder(params); + TopNResultBuilder resultBuilder = BaseTopNAlgorithm.makeResultBuilder(params, query); topNAlgorithm.run(params, resultBuilder, null); diff --git a/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java index c2baf13e3eb..267f2f278dd 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java +++ b/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java @@ -47,7 +47,9 @@ public interface TopNMetricSpec DateTime timestamp, DimensionSpec dimSpec, int threshold, - Comparator comparator + Comparator comparator, + List aggFactories, + List postAggs ); public byte[] getCacheKey(); @@ -55,4 +57,6 @@ public interface TopNMetricSpec public TopNMetricSpecBuilder configureOptimizer(TopNMetricSpecBuilder builder); public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector); + + public String getMetricName(DimensionSpec dimSpec); } diff --git a/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java index 9f6479baee4..4a40f4bb2d5 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java @@ -23,6 +23,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; import org.joda.time.DateTime; @@ -40,7 +41,8 @@ public class TopNNumericResultBuilder implements TopNResultBuilder private final DateTime timestamp; private final DimensionSpec dimSpec; private final String metricName; - + private final List aggFactories; + private final List postAggs; private MinMaxPriorityQueue pQueue = null; public TopNNumericResultBuilder( @@ -48,12 +50,16 @@ public class TopNNumericResultBuilder implements TopNResultBuilder DimensionSpec dimSpec, String metricName, int threshold, - final Comparator comparator + final Comparator comparator, + List aggFactories, + List postAggs ) { this.timestamp = timestamp; this.dimSpec = dimSpec; this.metricName = metricName; + this.aggFactories = aggFactories; + this.postAggs = AggregatorUtil.pruneDependentPostAgg(postAggs, this.metricName); instantiatePQueue(threshold, comparator); } @@ -62,9 +68,7 @@ public class TopNNumericResultBuilder implements TopNResultBuilder public TopNResultBuilder addEntry( String dimName, Object dimValIndex, - Object[] metricVals, - List aggFactories, - List postAggs + Object[] metricVals ) { Map metricValues = Maps.newLinkedHashMap(); @@ -75,6 +79,7 @@ public class TopNNumericResultBuilder implements TopNResultBuilder for (Object metricVal : metricVals) { metricValues.put(aggFactoryIter.next().getName(), metricVal); } + for (PostAggregator postAgg : postAggs) { metricValues.put(postAgg.getName(), postAgg.compute(metricValues)); } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java index 09a158b31de..1f3a8892733 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java @@ -21,7 +21,7 @@ package io.druid.query.topn; import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; +import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; import io.druid.collections.StupidPool; @@ -53,6 +53,12 @@ public class TopNQueryEngine public Iterable> query(final TopNQuery query, final StorageAdapter adapter) { + if (adapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + final List queryIntervals = query.getQuerySegmentSpec().getIntervals(); final Filter filter = Filters.convertDimensionFilters(query.getDimensionsFilter()); final QueryGranularity granularity = query.getGranularity(); @@ -62,10 +68,6 @@ public class TopNQueryEngine queryIntervals.size() == 1, "Can only handle a single interval, got[%s]", queryIntervals ); - if (mapFn == null) { - return Lists.newArrayList(); - } - return FunctionalIterable .create(adapter.makeCursors(filter, queryIntervals.get(0), granularity)) .transform( @@ -84,13 +86,6 @@ public class TopNQueryEngine private Function> getMapFn(TopNQuery query, final StorageAdapter adapter) { - if (adapter == null) { - log.warn( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped. Returning empty results." - ); - return null; - } - final Capabilities capabilities = adapter.getCapabilities(); final int cardinality = adapter.getDimensionCardinality(query.getDimensionSpec().getDimension()); int numBytesPerRecord = 0; diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index a7d77fde396..5db416f1d0b 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -46,6 +46,7 @@ import io.druid.query.Result; import io.druid.query.ResultGranularTimestampComparator; import io.druid.query.ResultMergeQueryRunner; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.PostAggregator; import io.druid.query.filter.DimFilter; @@ -64,11 +65,13 @@ import java.util.Map; public class TopNQueryQueryToolChest extends QueryToolChest, TopNQuery> { private static final byte TOPN_QUERY = 0x1; - private static final Joiner COMMA_JOIN = Joiner.on(","); - private static final TypeReference> TYPE_REFERENCE = new TypeReference>(){}; - - private static final TypeReference OBJECT_TYPE_REFERENCE = new TypeReference(){}; + private static final TypeReference> TYPE_REFERENCE = new TypeReference>() + { + }; + private static final TypeReference OBJECT_TYPE_REFERENCE = new TypeReference() + { + }; private final TopNQueryConfig config; @Inject @@ -137,7 +140,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest, Result> makeMetricManipulatorFn( + public Function, Result> makePreComputeManipulatorFn( final TopNQuery query, final MetricManipulationFn fn ) { @@ -146,7 +149,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest apply(@Nullable Result result) + public Result apply(Result result) { List> serializedValues = Lists.newArrayList( Iterables.transform( @@ -154,14 +157,19 @@ public class TopNQueryQueryToolChest extends QueryToolChest>() { @Override - public Map apply(@Nullable DimensionAndMetricValueExtractor input) + public Map apply(DimensionAndMetricValueExtractor input) { final Map values = Maps.newHashMap(); for (AggregatorFactory agg : query.getAggregatorSpecs()) { values.put(agg.getName(), fn.manipulate(agg, input.getMetric(agg.getName()))); } - for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { - values.put(postAgg.getName(), input.getMetric(postAgg.getName())); + for (PostAggregator postAgg : prunePostAggregators(query)) { + Object calculatedPostAgg = input.getMetric(postAgg.getName()); + if (calculatedPostAgg != null) { + values.put(postAgg.getName(), calculatedPostAgg); + } else { + values.put(postAgg.getName(), postAgg.compute(values)); + } } values.put(dimension, input.getDimensionValue(dimension)); @@ -179,6 +187,60 @@ public class TopNQueryQueryToolChest extends QueryToolChest, Result> makePostComputeManipulatorFn( + final TopNQuery query, final MetricManipulationFn fn + ) + { + return new Function, Result>() + { + private String dimension = query.getDimensionSpec().getOutputName(); + + @Override + public Result apply(Result result) + { + List> serializedValues = Lists.newArrayList( + Iterables.transform( + result.getValue(), + new Function>() + { + @Override + public Map apply(DimensionAndMetricValueExtractor input) + { + final Map values = Maps.newHashMap(); + // put non finalized aggregators for calculating dependent post Aggregators + for (AggregatorFactory agg : query.getAggregatorSpecs()) { + values.put(agg.getName(), input.getMetric(agg.getName())); + } + + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + Object calculatedPostAgg = input.getMetric(postAgg.getName()); + if (calculatedPostAgg != null) { + values.put(postAgg.getName(), calculatedPostAgg); + } else { + values.put(postAgg.getName(), postAgg.compute(values)); + } + } + for (AggregatorFactory agg : query.getAggregatorSpecs()) { + values.put(agg.getName(), fn.manipulate(agg, input.getMetric(agg.getName()))); + } + + values.put(dimension, input.getDimensionValue(dimension)); + + return values; + } + } + ) + ); + + return new Result( + result.getTimestamp(), + new TopNResultValue(serializedValues) + ); + } + }; + } + @Override public TypeReference> getResultTypeReference() { @@ -191,7 +253,11 @@ public class TopNQueryQueryToolChest extends QueryToolChest, Object, TopNQuery>() { private final List aggs = query.getAggregatorSpecs(); - private final List postAggs = query.getPostAggregatorSpecs(); + private final List postAggs = AggregatorUtil.pruneDependentPostAgg( + query.getPostAggregatorSpecs(), + query.getTopNMetricSpec() + .getMetricName(query.getDimensionSpec()) + ); @Override public byte[] computeCacheKey(TopNQuery query) @@ -231,7 +297,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest, Object>() { @Override - public Object apply(@Nullable final Result input) + public Object apply(final Result input) { List results = Lists.newArrayList(input.getValue()); final List retVal = Lists.newArrayListWithCapacity(results.size() + 1); @@ -259,7 +325,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest apply(@Nullable Object input) + public Result apply(Object input) { List results = (List) input; List> retVal = Lists.newArrayListWithCapacity(results.size()); @@ -313,6 +379,11 @@ public class TopNQueryQueryToolChest extends QueryToolChest> getOrdering() + { + return Ordering.natural(); + } + private static class ThresholdAdjustingQueryRunner implements QueryRunner> { private final QueryRunner> runner; @@ -339,7 +410,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest, Result>() { @Override - public Result apply(@Nullable Result input) + public Result apply(Result input) { return new Result( input.getTimestamp(), @@ -398,8 +469,11 @@ public class TopNQueryQueryToolChest extends QueryToolChest> getOrdering() + private static List prunePostAggregators(TopNQuery query) { - return Ordering.natural(); + return AggregatorUtil.pruneDependentPostAgg( + query.getPostAggregatorSpecs(), + query.getTopNMetricSpec().getMetricName(query.getDimensionSpec()) + ); } } diff --git a/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java index 5823ee3eece..97b20175380 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java @@ -33,9 +33,7 @@ public interface TopNResultBuilder public TopNResultBuilder addEntry( String dimName, Object dimValIndex, - Object[] metricVals, - List aggFactories, - List postAggs + Object[] metricVals ); public TopNResultBuilder addEntry( diff --git a/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java b/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java index 965859fecfc..e79259554ac 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java +++ b/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java @@ -23,6 +23,7 @@ package io.druid.data.input; import com.google.protobuf.AbstractMessage; +import com.google.protobuf.UnknownFieldSet; public final class ProtoTestEventWrapper { private ProtoTestEventWrapper() {} @@ -85,7 +86,13 @@ public final class ProtoTestEventWrapper { public ProtoTestEvent getDefaultInstanceForType() { return defaultInstance; } - + + @Override + public UnknownFieldSet getUnknownFields() + { + return UnknownFieldSet.getDefaultInstance(); + } + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; @@ -1049,6 +1056,7 @@ public final class ProtoTestEventWrapper { new com.google.protobuf.Descriptors.FileDescriptor[] { }, assigner); } + // @@protoc_insertion_point(outer_class_scope) } diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 7ca66b53d7e..1e2b6b4601d 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -25,6 +25,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; import io.druid.query.aggregation.post.ConstantPostAggregator; @@ -60,6 +61,7 @@ public class QueryRunnerTestHelper public static final String indexMetric = "index"; public static final String uniqueMetric = "uniques"; public static final String addRowsIndexConstantMetric = "addRowsIndexConstant"; + public static String dependentPostAggMetric = "dependentPostAgg"; public static final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); public static final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); public static final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); @@ -72,8 +74,26 @@ public class QueryRunnerTestHelper public static final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); public static final ArithmeticPostAggregator addRowsIndexConstant = new ArithmeticPostAggregator( - "addRowsIndexConstant", "+", Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) + addRowsIndexConstantMetric, "+", Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) ); + // dependent on AddRowsIndexContact postAgg + public static final ArithmeticPostAggregator dependentPostAgg = new ArithmeticPostAggregator( + dependentPostAggMetric, + "+", + Lists.newArrayList( + constant, + new FieldAccessPostAggregator(addRowsIndexConstantMetric, addRowsIndexConstantMetric), + new FieldAccessPostAggregator("rows", "rows") + ) + ); + + public static final String hyperUniqueFinalizingPostAggMetric = "hyperUniqueFinalizingPostAggMetric"; + public static ArithmeticPostAggregator hyperUniqueFinalizingPostAgg = new ArithmeticPostAggregator( + hyperUniqueFinalizingPostAggMetric, + "+", + Lists.newArrayList(new HyperUniqueFinalizingPostAggregator(uniqueMetric), new ConstantPostAggregator(null, 1, 1)) + ); + public static final List commonAggregators = Arrays.asList( rowsCount, indexDoubleSum, @@ -151,7 +171,10 @@ public class QueryRunnerTestHelper ) { return new FinalizeResultsQueryRunner( - factory.createRunner(adapter), + new BySegmentQueryRunner( + segmentId, adapter.getDataInterval().getStart(), + factory.createRunner(adapter) + ), factory.getToolchest() ); } diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java b/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java new file mode 100644 index 00000000000..08c51f9438b --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java @@ -0,0 +1,217 @@ +/* + * 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.query.aggregation; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.metamx.common.Pair; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static io.druid.query.QueryRunnerTestHelper.dependentPostAggMetric; + +public class AggregatorUtilTest +{ + + @Test + public void testPruneDependentPostAgg() + { + PostAggregator agg1 = new ArithmeticPostAggregator( + "abc", "+", Lists.newArrayList( + new ConstantPostAggregator("1", 1L, 1L), new ConstantPostAggregator("2", 2L, 2L) + ) + ); + PostAggregator dependency1 = new ArithmeticPostAggregator( + "dep1", "+", Lists.newArrayList( + new ConstantPostAggregator("1", 1L, 1L), new ConstantPostAggregator("4", 4L, 4L) + ) + ); + PostAggregator agg2 = new FieldAccessPostAggregator("def", "def"); + PostAggregator dependency2 = new FieldAccessPostAggregator("dep2", "dep2"); + PostAggregator aggregator = new ArithmeticPostAggregator( + "finalAgg", + "+", + Lists.newArrayList( + new FieldAccessPostAggregator("dep1", "dep1"), + new FieldAccessPostAggregator("dep2", "dep2") + ) + ); + List prunedAgg = AggregatorUtil.pruneDependentPostAgg( + Lists.newArrayList( + agg1, + dependency1, + agg2, + dependency2, + aggregator + ), aggregator.getName() + ); + Assert.assertEquals(Lists.newArrayList(dependency1, dependency2, aggregator), prunedAgg); + } + + @Test + public void testOutOfOrderPruneDependentPostAgg() + { + PostAggregator agg1 = new ArithmeticPostAggregator( + "abc", "+", Lists.newArrayList( + new ConstantPostAggregator("1", 1L, 1L), new ConstantPostAggregator("2", 2L, 2L) + ) + ); + PostAggregator dependency1 = new ArithmeticPostAggregator( + "dep1", "+", Lists.newArrayList( + new ConstantPostAggregator("1", 1L, 1L), new ConstantPostAggregator("4", 4L, 4L) + ) + ); + PostAggregator agg2 = new FieldAccessPostAggregator("def", "def"); + PostAggregator dependency2 = new FieldAccessPostAggregator("dep2", "dep2"); + PostAggregator aggregator = new ArithmeticPostAggregator( + "finalAgg", + "+", + Lists.newArrayList( + new FieldAccessPostAggregator("dep1", "dep1"), + new FieldAccessPostAggregator("dep2", "dep2") + ) + ); + List prunedAgg = AggregatorUtil.pruneDependentPostAgg( + Lists.newArrayList( + agg1, + dependency1, + aggregator, // dependency is added later than the aggregator + agg2, + dependency2 + ), aggregator.getName() + ); + Assert.assertEquals(Lists.newArrayList(dependency1, aggregator), prunedAgg); + } + + @Test + public void testCondenseAggregators() + { + + ArrayList aggregatorFactories = Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ); + + List postAggregatorList = Arrays.asList( + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg + ); + Pair, List> aggregatorsPair = AggregatorUtil.condensedAggregators( + aggregatorFactories, + postAggregatorList, + dependentPostAggMetric + ); + // verify aggregators + Assert.assertEquals( + Lists.newArrayList(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.indexDoubleSum), + aggregatorsPair.lhs + ); + Assert.assertEquals( + Lists.newArrayList( + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg + ), aggregatorsPair.rhs + ); + + } + + @Test + public void testNullPostAggregatorNames() + { + AggregatorFactory agg1 = new DoubleSumAggregatorFactory("agg1", "value"); + AggregatorFactory agg2 = new DoubleSumAggregatorFactory("agg2", "count"); + PostAggregator postAgg1 = new ArithmeticPostAggregator( + null, "*", Lists.newArrayList( + new FieldAccessPostAggregator( + null, + "agg1" + ), new FieldAccessPostAggregator(null, "agg2") + ) + ); + + PostAggregator postAgg2 = new ArithmeticPostAggregator( + "postAgg", + "/", + Lists.newArrayList( + new FieldAccessPostAggregator( + null, + "agg1" + ), new FieldAccessPostAggregator(null, "agg2") + ) + ); + + Assert.assertEquals( + new Pair(Lists.newArrayList(agg1, agg2), Lists.newArrayList(postAgg2)), AggregatorUtil.condensedAggregators( + Lists.newArrayList(agg1, agg2), + Lists.newArrayList(postAgg1, postAgg2), + "postAgg" + ) + ); + + } + + @Test + public void testCasing() + { + AggregatorFactory agg1 = new DoubleSumAggregatorFactory("Agg1", "value"); + AggregatorFactory agg2 = new DoubleSumAggregatorFactory("Agg2", "count"); + PostAggregator postAgg1 = new ArithmeticPostAggregator( + null, "*", Lists.newArrayList( + new FieldAccessPostAggregator( + null, + "Agg1" + ), new FieldAccessPostAggregator(null, "Agg2") + ) + ); + + PostAggregator postAgg2 = new ArithmeticPostAggregator( + "postAgg", + "/", + Lists.newArrayList( + new FieldAccessPostAggregator( + null, + "Agg1" + ), new FieldAccessPostAggregator(null, "Agg2") + ) + ); + + Assert.assertEquals( + new Pair(Lists.newArrayList(agg1, agg2), Lists.newArrayList(postAgg2)), AggregatorUtil.condensedAggregators( + Lists.newArrayList(agg1, agg2), + Lists.newArrayList(postAgg1, postAgg2), + "postAgg" + ) + ); + } + +} diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index f336a26884e..97e64a0ec0c 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -35,6 +35,7 @@ import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; +import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; @@ -45,6 +46,8 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.MaxAggregatorFactory; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; +import io.druid.query.dimension.ExtractionDimensionSpec; +import io.druid.query.extraction.RegexDimExtractionFn; import io.druid.query.filter.JavaScriptDimFilter; import io.druid.query.filter.RegexDimFilter; import io.druid.query.groupby.having.EqualToHavingSpec; @@ -61,7 +64,6 @@ import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -179,39 +181,118 @@ public class GroupByQueryRunnerTest } @Test - public void testGroupByWithTimeZone() + public void testGroupByWithUniques() { - DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles"); - - GroupByQuery query = GroupByQuery.builder() + GroupByQuery query = GroupByQuery + .builder() .setDataSource(QueryRunnerTestHelper.dataSource) - .setInterval("2011-03-31T00:00:00-07:00/2011-04-02T00:00:00-07:00") + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.qualityUniques + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow( + "2011-04-01", + "rows", + 26L, + "uniques", + QueryRunnerTestHelper.UNIQUES_9 + ) + ); + + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByWithDimExtractionFn() + { + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions( - Lists.newArrayList( - (DimensionSpec) new DefaultDimensionSpec( + Lists.newArrayList( + new ExtractionDimensionSpec( "quality", - "alias" + "alias", + new RegexDimExtractionFn("(\\w{1})") ) ) ) .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory( - "idx", - "index" - ) - ) - ) - .setGranularity( - new PeriodGranularity( - new Period("P1D"), - null, - tz + new LongSumAggregatorFactory("idx", "index") ) ) + .setGranularity(QueryRunnerTestHelper.dayGran) .build(); + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "a", "rows", 1L, "idx", 135L), + createExpectedRow("2011-04-01", "alias", "b", "rows", 1L, "idx", 118L), + createExpectedRow("2011-04-01", "alias", "e", "rows", 1L, "idx", 158L), + createExpectedRow("2011-04-01", "alias", "h", "rows", 1L, "idx", 120L), + createExpectedRow("2011-04-01", "alias", "m", "rows", 3L, "idx", 2870L), + createExpectedRow("2011-04-01", "alias", "n", "rows", 1L, "idx", 121L), + createExpectedRow("2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L), + createExpectedRow("2011-04-01", "alias", "t", "rows", 2L, "idx", 197L), + + createExpectedRow("2011-04-02", "alias", "a", "rows", 1L, "idx", 147L), + createExpectedRow("2011-04-02", "alias", "b", "rows", 1L, "idx", 112L), + createExpectedRow("2011-04-02", "alias", "e", "rows", 1L, "idx", 166L), + createExpectedRow("2011-04-02", "alias", "h", "rows", 1L, "idx", 113L), + createExpectedRow("2011-04-02", "alias", "m", "rows", 3L, "idx", 2447L), + createExpectedRow("2011-04-02", "alias", "n", "rows", 1L, "idx", 114L), + createExpectedRow("2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L), + createExpectedRow("2011-04-02", "alias", "t", "rows", 2L, "idx", 223L) + ); + + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByWithTimeZone() + { + DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles"); + + GroupByQuery query = GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-03-31T00:00:00-07:00/2011-04-02T00:00:00-07:00") + .setDimensions( + Lists.newArrayList( + (DimensionSpec) new DefaultDimensionSpec( + "quality", + "alias" + ) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ) + ) + ) + .setGranularity( + new PeriodGranularity( + new Period("P1D"), + null, + tz + ) + ) + .build(); + List expectedResults = Arrays.asList( createExpectedRow(new DateTime("2011-03-31", tz), "alias", "automotive", "rows", 1L, "idx", 135L), createExpectedRow(new DateTime("2011-03-31", tz), "alias", "business", "rows", 1L, "idx", 118L), @@ -949,8 +1030,14 @@ public class GroupByQueryRunnerTest private Iterable runQuery(GroupByQuery query) { - QueryToolChest toolChest = factory.getToolchest(); - Sequence queryResult = toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)).run(query); + + QueryToolChest toolChest = factory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)), + toolChest + ); + + Sequence queryResult = theRunner.run(query); return Sequences.toList(queryResult, Lists.newArrayList()); } diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryTest.java index 1dd50e9493d..b88f290984b 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryTest.java @@ -22,6 +22,7 @@ package io.druid.query.timeboundary; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; import io.druid.query.Query; @@ -38,8 +39,8 @@ public class TimeBoundaryQueryTest public void testQuerySerialization() throws IOException { Query query = Druids.newTimeBoundaryQueryBuilder() - .dataSource("testing") - .build(); + .dataSource("testing") + .build(); String json = jsonMapper.writeValueAsString(query); Query serdeQuery = jsonMapper.readValue(json, Query.class); @@ -47,4 +48,79 @@ public class TimeBoundaryQueryTest Assert.assertEquals(query, serdeQuery); } + @Test + public void testContextSerde() throws Exception + { + final TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() + .dataSource("foo") + .intervals("2013/2014") + .context( + ImmutableMap.of( + "priority", + 1, + "useCache", + true, + "populateCache", + true, + "finalize", + true + ) + ).build(); + + final ObjectMapper mapper = new DefaultObjectMapper(); + + final TimeBoundaryQuery serdeQuery = mapper.readValue( + mapper.writeValueAsBytes( + mapper.readValue( + mapper.writeValueAsString( + query + ), TimeBoundaryQuery.class + ) + ), TimeBoundaryQuery.class + ); + + + Assert.assertEquals(1, serdeQuery.getContextValue("priority")); + Assert.assertEquals(true, serdeQuery.getContextValue("useCache")); + Assert.assertEquals(true, serdeQuery.getContextValue("populateCache")); + Assert.assertEquals(true, serdeQuery.getContextValue("finalize")); + } + + @Test + public void testContextSerde2() throws Exception + { + final TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() + .dataSource("foo") + .intervals("2013/2014") + .context( + ImmutableMap.of( + "priority", + "1", + "useCache", + "true", + "populateCache", + "true", + "finalize", + "true" + ) + ).build(); + + final ObjectMapper mapper = new DefaultObjectMapper(); + + final TimeBoundaryQuery serdeQuery = mapper.readValue( + mapper.writeValueAsBytes( + mapper.readValue( + mapper.writeValueAsString( + query + ), TimeBoundaryQuery.class + ) + ), TimeBoundaryQuery.class + ); + + + Assert.assertEquals("1", serdeQuery.getContextValue("priority")); + Assert.assertEquals("true", serdeQuery.getContextValue("useCache")); + Assert.assertEquals("true", serdeQuery.getContextValue("populateCache")); + Assert.assertEquals("true", serdeQuery.getContextValue("finalize")); + } } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java index e574cbd4b44..ff3266f303c 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java @@ -20,16 +20,11 @@ package io.druid.query.timeseries; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import io.druid.granularity.QueryGranularity; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; -import io.druid.query.aggregation.PostAggregator; -import io.druid.query.aggregation.post.ArithmeticPostAggregator; -import io.druid.query.aggregation.post.ConstantPostAggregator; -import io.druid.query.aggregation.post.FieldAccessPostAggregator; import junit.framework.Assert; import org.joda.time.DateTime; import org.junit.Test; @@ -43,21 +38,10 @@ public class TimeseriesBinaryFnTest { final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); - final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); - final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); - final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); - final ArithmeticPostAggregator addRowsIndexConstant = new ArithmeticPostAggregator( - "addRowsIndexConstant", - "+", - Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) - ); final List aggregatorFactories = Arrays.asList( rowsCount, indexLongSum ); - final List postAggregators = Arrays.asList( - addRowsIndexConstant - ); final DateTime currTime = new DateTime(); @Test @@ -87,16 +71,14 @@ public class TimeseriesBinaryFnTest new TimeseriesResultValue( ImmutableMap.of( "rows", 3L, - "index", 5L, - "addRowsIndexConstant", 9.0 + "index", 5L ) ) ); Result actual = new TimeseriesBinaryFn( QueryGranularity.ALL, - aggregatorFactories, - postAggregators + aggregatorFactories ).apply( result1, result2 @@ -131,16 +113,14 @@ public class TimeseriesBinaryFnTest new TimeseriesResultValue( ImmutableMap.of( "rows", 3L, - "index", 5L, - "addRowsIndexConstant", 9.0 + "index", 5L ) ) ); Result actual = new TimeseriesBinaryFn( QueryGranularity.DAY, - aggregatorFactories, - postAggregators + aggregatorFactories ).apply( result1, result2 @@ -166,8 +146,7 @@ public class TimeseriesBinaryFnTest Result actual = new TimeseriesBinaryFn( QueryGranularity.ALL, - aggregatorFactories, - postAggregators + aggregatorFactories ).apply( result1, result2 @@ -202,16 +181,14 @@ public class TimeseriesBinaryFnTest new TimeseriesResultValue( ImmutableMap.of( "rows", 3L, - "index", 5L, - "addRowsIndexConstant", 9.0 + "index", 5L ) ) ); Result actual = new TimeseriesBinaryFn( QueryGranularity.ALL, - aggregatorFactories, - postAggregators + aggregatorFactories ).apply( result1, result2 diff --git a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java new file mode 100644 index 00000000000..65ab16ad19b --- /dev/null +++ b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java @@ -0,0 +1,135 @@ +/* + * 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.query.topn; + +import com.google.caliper.Param; +import com.google.caliper.Runner; +import com.google.caliper.SimpleBenchmark; +import com.google.common.collect.Lists; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Result; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; +import org.joda.time.DateTime; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TopNBinaryFnBenchmark extends SimpleBenchmark +{ + @Param({"1", "5", "10", "15"}) + int aggCount; + @Param({"1", "5", "10", "15"}) + int postAggCount; + @Param({"1000", "10000"}) + int threshold; + Result result1; + Result result2; + TopNBinaryFn fn; + + public static void main(String[] args) throws Exception + { + Runner.main(TopNBinaryFnBenchmark.class, args); + } + + @Override + protected void setUp() throws Exception + { + + final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); + final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); + final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); + + + final List aggregatorFactories = new ArrayList<>(); + aggregatorFactories.add(new CountAggregatorFactory("rows")); + aggregatorFactories.add(new LongSumAggregatorFactory("index", "index")); + for (int i = 1; i < aggCount; i++) { + aggregatorFactories.add(new CountAggregatorFactory("rows" + i)); + } + final List postAggregators = new ArrayList<>(); + for (int i = 0; i < postAggCount; i++) { + postAggregators.add( + new ArithmeticPostAggregator( + "addrowsindexconstant" + i, + "+", + Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) + ) + ); + } + final DateTime currTime = new DateTime(); + List> list = new ArrayList<>(); + for (int i = 0; i < threshold; i++) { + Map res = new HashMap<>(); + res.put("testdim", "" + i); + res.put("rows", 1L); + for (int j = 0; j < aggCount; j++) { + res.put("rows" + j, 1L); + } + res.put("index", 1L); + list.add(res); + } + result1 = new Result<>( + currTime, + new TopNResultValue(list) + ); + + List> list2 = new ArrayList<>(); + for (int i = 0; i < threshold; i++) { + Map res = new HashMap<>(); + res.put("testdim", "" + i); + res.put("rows", 2L); + for (int j = 0; j < aggCount; j++) { + res.put("rows" + j, 2L); + } + res.put("index", 2L); + list2.add(res); + } + result2 = new Result<>( + currTime, + new TopNResultValue(list2) + ); + fn = new TopNBinaryFn( + TopNResultMerger.identity, + QueryGranularity.ALL, + new DefaultDimensionSpec("testdim", null), + new NumericTopNMetricSpec("index"), + 100, + aggregatorFactories, + postAggregators + ); + } + + public void timeMerge(int nReps) + { + for (int i = 0; i < nReps; i++) { + fn.apply(result1, result2); + } + } + +} diff --git a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java index 0eba9778c4e..c8958dc6580 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java @@ -129,15 +129,13 @@ public class TopNBinaryFnTest ImmutableMap.of( "testdim", "1", "rows", 3L, - "index", 5L, - "addrowsindexconstant", 9.0 + "index", 5L ), ImmutableMap.of( "testdim", "2", "rows", 4L, - "index", 4L, - "addrowsindexconstant", 9.0 + "index", 4L ) ) ) @@ -214,14 +212,12 @@ public class TopNBinaryFnTest ImmutableMap.of( "testdim", "1", "rows", 3L, - "index", 5L, - "addrowsindexconstant", 9.0 + "index", 5L ), ImmutableMap.of( "testdim", "2", "rows", 4L, - "index", 4L, - "addrowsindexconstant", 9.0 + "index", 4L ) ) ) @@ -298,17 +294,20 @@ public class TopNBinaryFnTest ImmutableMap.of( "rows", 1L, "index", 2L, - "testdim", "1" + "testdim", "1", + "addrowsindexconstant", 3.0 ), ImmutableMap.of( "rows", 2L, "index", 4L, - "testdim", "2" + "testdim", "2", + "addrowsindexconstant", 7.0 ), ImmutableMap.of( "rows", 0L, "index", 2L, - "testdim", "3" + "testdim", "3", + "addrowsindexconstant", 3.0 ) ) ) @@ -320,17 +319,20 @@ public class TopNBinaryFnTest ImmutableMap.of( "rows", 2L, "index", 3L, - "testdim", "1" + "testdim", "1", + "addrowsindexconstant", 6.0 ), ImmutableMap.of( "rows", 2L, "index", 0L, - "testdim", "2" + "testdim", "2", + "addrowsindexconstant", 3.0 ), ImmutableMap.of( - "rows", 0L, - "index", 1L, - "testdim", "3" + "rows", 4L, + "index", 5L, + "testdim", "other", + "addrowsindexconstant", 10.0 ) ) ) @@ -340,6 +342,12 @@ public class TopNBinaryFnTest currTime, new TopNResultValue( ImmutableList.>of( + ImmutableMap.of( + "testdim", "other", + "rows", 4L, + "index", 5L, + "addrowsindexconstant", 10.0 + ), ImmutableMap.of( "testdim", "1", "rows", 3L, @@ -361,7 +369,7 @@ public class TopNBinaryFnTest QueryGranularity.ALL, new DefaultDimensionSpec("testdim", null), new NumericTopNMetricSpec("addrowsindexconstant"), - 2, + 3, aggregatorFactories, postAggregators ).apply( @@ -427,14 +435,12 @@ public class TopNBinaryFnTest ImmutableMap.of( "testdim", "1", "rows", 3L, - "index", 5L, - "addrowsindexconstant", 9.0 + "index", 5L ), ImmutableMap.of( "testdim", "2", "rows", 4L, - "index", 4L, - "addrowsindexconstant", 9.0 + "index", 4L ) ) ) diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 291eab8171a..c8628bfdca5 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; +import io.druid.query.BySegmentResultValueClass; import io.druid.query.Druids; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; @@ -64,7 +65,7 @@ public class TopNQueryRunnerTest { List retVal = Lists.newArrayList(); retVal.addAll( - TopNQueryRunnerTestHelper.makeQueryRunners( + QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( TestQueryRunners.getPool(), new TopNQueryQueryToolChest(new TopNQueryConfig()) @@ -72,7 +73,7 @@ public class TopNQueryRunnerTest ) ); retVal.addAll( - TopNQueryRunnerTestHelper.makeQueryRunners( + QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( new StupidPool( new Supplier() @@ -698,13 +699,13 @@ public class TopNQueryRunnerTest .fields( Lists.newArrayList( Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) - .value("billyblank") - .build(), + .dimension(providerDimension) + .value("billyblank") + .build(), Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.qualityDimension) - .value("mezzanine") - .build() + .dimension(QueryRunnerTestHelper.qualityDimension) + .value("mezzanine") + .build() ) ).build(); TopNQuery query = new TopNQueryBuilder() @@ -1176,4 +1177,170 @@ public class TopNQueryRunnerTest TestHelper.assertExpectedResults(expectedResults, runner.run(query)); } + + @Test + public void testTopNDependentPostAgg() { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(providerDimension) + .metric(QueryRunnerTestHelper.dependentPostAggMetric) + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators( + Arrays.asList( + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg, + QueryRunnerTestHelper.hyperUniqueFinalizingPostAgg + ) + ) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put(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( + QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, + QueryRunnerTestHelper.UNIQUES_2 + 1.0 + ) + .build(), + ImmutableMap.builder() + .put(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( + QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, + QueryRunnerTestHelper.UNIQUES_2 + 1.0 + ) + .build(), + ImmutableMap.builder() + .put(providerDimension, "spot") + .put("rows", 837L) + .put("index", 95606.57232284546D) + .put("addRowsIndexConstant", 96444.57232284546D) + .put(QueryRunnerTestHelper.dependentPostAggMetric, 97282.57232284546D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put( + QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, + QueryRunnerTestHelper.UNIQUES_9 + 1.0 + ) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .build() + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNBySegmentResults() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(QueryRunnerTestHelper.providerDimension) + .metric(QueryRunnerTestHelper.dependentPostAggMetric) + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators( + Arrays.asList( + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg + ) + ) + .context(ImmutableMap.of("finalize", true, "bySegment", true)) + .build(); + TopNResultValue topNResult = new TopNResultValue( + Arrays.>asList( + ImmutableMap.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) + .build(), + ImmutableMap.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) + .build(), + ImmutableMap.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) + .build() + ) + ); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new BySegmentResultValueClass( + Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + topNResult + ) + ), + QueryRunnerTestHelper.segmentId, + new Interval("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z") + ) + ) + ); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } } diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTestHelper.java deleted file mode 100644 index 97b837a4b48..00000000000 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTestHelper.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.topn; - -import io.druid.query.FinalizeResultsQueryRunner; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.QueryRunnerFactory; -import io.druid.segment.IncrementalIndexSegment; -import io.druid.segment.QueryableIndex; -import io.druid.segment.QueryableIndexSegment; -import io.druid.segment.Segment; -import io.druid.segment.TestIndex; -import io.druid.segment.incremental.IncrementalIndex; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collection; - -public class TopNQueryRunnerTestHelper -{ - @SuppressWarnings("unchecked") - public static Collection makeQueryRunners( - QueryRunnerFactory factory - ) - throws IOException - { - final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(); - final QueryableIndex mMappedTestIndex = TestIndex.getMMappedTestIndex(); - final QueryableIndex mergedRealtimeIndex = TestIndex.mergedRealtimeIndex(); - return Arrays.asList( - new Object[][]{ - { - makeQueryRunner(factory, new IncrementalIndexSegment(rtIndex, null)) - }, - { - makeQueryRunner(factory, new QueryableIndexSegment(null, mMappedTestIndex)) - }, - { - makeQueryRunner(factory, new QueryableIndexSegment(null, mergedRealtimeIndex)) - } - } - ); - } - - public static QueryRunner makeQueryRunner( - QueryRunnerFactory> factory, - Segment adapter - ) - { - return new FinalizeResultsQueryRunner( - factory.createRunner(adapter), - factory.getToolchest() - ); - } -} \ No newline at end of file diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 66d37cf381c..1635c984c4c 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 06f414ecc85..78424232c00 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java index 69b1843c125..e8e98987bf1 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java @@ -39,7 +39,7 @@ public class S3DataSegmentArchiver extends S3DataSegmentMover implements DataSeg S3DataSegmentPusherConfig restoreConfig ) { - super(s3Client); + super(s3Client, restoreConfig); this.archiveConfig = archiveConfig; this.restoreConfig = restoreConfig; } diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index fbf18df4c18..d259ab185d2 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -30,6 +30,7 @@ import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import org.jets3t.service.ServiceException; +import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Object; @@ -41,13 +42,16 @@ public class S3DataSegmentMover implements DataSegmentMover private static final Logger log = new Logger(S3DataSegmentMover.class); private final RestS3Service s3Client; + private final S3DataSegmentPusherConfig config; @Inject public S3DataSegmentMover( - RestS3Service s3Client + RestS3Service s3Client, + S3DataSegmentPusherConfig config ) { this.s3Client = s3Client; + this.config = config; } @Override @@ -124,7 +128,11 @@ public class S3DataSegmentMover implements DataSegmentMover targetS3Bucket, targetS3Path ); - s3Client.moveObject(s3Bucket, s3Path, targetS3Bucket, new S3Object(targetS3Path), false); + final S3Object target = new S3Object(targetS3Path); + if(!config.getDisableAcl()) { + target.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + } + s3Client.moveObject(s3Bucket, s3Path, targetS3Bucket, target, false); } } else { // ensure object exists in target location diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java index 664c270799b..5dad247544b 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java @@ -89,7 +89,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher toPush.setBucketName(outputBucket); toPush.setKey(s3Path); if (!config.getDisableAcl()) { - toPush.setAcl(AccessControlList.REST_CANNED_AUTHENTICATED_READ); + toPush.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); } log.info("Pushing %s.", toPush); diff --git a/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java b/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java index 6206da881a4..c13d22de5f1 100644 --- a/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java +++ b/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java @@ -62,7 +62,7 @@ public class S3DataSegmentMoverTest public void testMove() throws Exception { MockStorageService mockS3Client = new MockStorageService(); - S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client); + S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); mockS3Client.putObject("main", new S3Object("baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip")); mockS3Client.putObject("main", new S3Object("baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/descriptor.json")); @@ -82,7 +82,7 @@ public class S3DataSegmentMoverTest public void testMoveNoop() throws Exception { MockStorageService mockS3Client = new MockStorageService(); - S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client); + S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); mockS3Client.putObject("archive", new S3Object("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip")); mockS3Client.putObject("archive", new S3Object("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/descriptor.json")); @@ -103,7 +103,7 @@ public class S3DataSegmentMoverTest public void testMoveException() throws Exception { MockStorageService mockS3Client = new MockStorageService(); - S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client); + S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); mover.move( sourceSegment, diff --git a/server/pom.xml b/server/pom.xml index 34c9244f0ef..ca5e8fe9580 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/server/src/main/java/io/druid/client/CachePopulatingQueryRunner.java b/server/src/main/java/io/druid/client/CachePopulatingQueryRunner.java deleted file mode 100644 index a9997946542..00000000000 --- a/server/src/main/java/io/druid/client/CachePopulatingQueryRunner.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 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.client; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Lists; -import com.metamx.common.guava.Accumulator; -import com.metamx.common.guava.Sequence; -import com.metamx.common.guava.Sequences; -import io.druid.client.cache.Cache; -import io.druid.client.cache.CacheConfig; -import io.druid.client.cache.MapCache; -import io.druid.query.CacheStrategy; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.QueryToolChest; -import io.druid.query.SegmentDescriptor; - -import java.util.ArrayList; - -public class CachePopulatingQueryRunner implements QueryRunner -{ - - private final String segmentIdentifier; - private final SegmentDescriptor segmentDescriptor; - private final QueryRunner base; - private final QueryToolChest toolChest; - private final Cache cache; - private final ObjectMapper mapper; - private final CacheConfig cacheConfig; - - public CachePopulatingQueryRunner( - String segmentIdentifier, - SegmentDescriptor segmentDescriptor, ObjectMapper mapper, - Cache cache, - QueryToolChest toolchest, - QueryRunner base, - CacheConfig cacheConfig - ) - { - this.base = base; - this.segmentIdentifier = segmentIdentifier; - this.segmentDescriptor = segmentDescriptor; - this.toolChest = toolchest; - this.cache = cache; - this.mapper = mapper; - this.cacheConfig = cacheConfig; - } - - @Override - public Sequence run(Query query) - { - - final CacheStrategy strategy = toolChest.getCacheStrategy(query); - - final boolean populateCache = Boolean.parseBoolean(query.getContextValue(CacheConfig.POPULATE_CACHE, "true")) - && strategy != null - && cacheConfig.isPopulateCache() - // historical only populates distributed cache since the cache lookups are done at broker. - && !(cache instanceof MapCache); - if (populateCache) { - Sequence results = base.run(query); - Cache.NamedKey key = CacheUtil.computeSegmentCacheKey( - segmentIdentifier, - segmentDescriptor, - strategy.computeCacheKey(query) - ); - ArrayList resultAsList = Sequences.toList(results, new ArrayList()); - CacheUtil.populate( - cache, - mapper, - key, - Lists.transform(resultAsList, strategy.prepareForCache()) - ); - return Sequences.simple(resultAsList); - } else { - return base.run(query); - } - } - -} diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 65ac6bea415..da1857bea47 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -31,6 +31,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; import com.metamx.common.Pair; @@ -62,7 +63,6 @@ import io.druid.timeline.partition.PartitionChunk; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -125,24 +125,24 @@ public class CachingClusteredClient implements QueryRunner final List> cachedResults = Lists.newArrayList(); final Map cachePopulatorMap = Maps.newHashMap(); - final boolean useCache = Boolean.parseBoolean(query.getContextValue(CacheConfig.USE_CACHE, "true")) + final boolean useCache = query.getContextUseCache(true) && strategy != null && cacheConfig.isUseCache(); - final boolean populateCache = Boolean.parseBoolean(query.getContextValue(CacheConfig.POPULATE_CACHE, "true")) + final boolean populateCache = query.getContextPopulateCache(true) && strategy != null && cacheConfig.isPopulateCache(); - final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment", "false")); + final boolean isBySegment = query.getContextBySegment(false); ImmutableMap.Builder contextBuilder = new ImmutableMap.Builder<>(); - final String priority = query.getContextValue("priority", "0"); + final int priority = query.getContextPriority(0); contextBuilder.put("priority", priority); if (populateCache) { - contextBuilder.put(CacheConfig.POPULATE_CACHE, "false"); - contextBuilder.put("bySegment", "true"); + contextBuilder.put(CacheConfig.POPULATE_CACHE, false); + contextBuilder.put("bySegment", true); } - contextBuilder.put("intermediate", "true"); + contextBuilder.put("intermediate", true); final Query rewrittenQuery = query.withOverriddenContext(contextBuilder.build()); @@ -333,27 +333,31 @@ public class CachingClusteredClient implements QueryRunner clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec)), new Function>() { - private final Function prepareForCache = strategy.prepareForCache(); + private final Function cacheFn = strategy.prepareForCache(); @Override public Sequence apply(Object input) { Result result = (Result) input; final BySegmentResultValueClass value = (BySegmentResultValueClass) result.getValue(); - String segmentIdentifier = value.getSegmentId(); - final Iterable segmentResults = value.getResults(); - CachePopulator cachePopulator = cachePopulatorMap.get( - String.format("%s_%s", segmentIdentifier, value.getInterval()) - ); - if (cachePopulator != null) { - cachePopulator.populate(Iterables.transform(segmentResults, prepareForCache)); - } + final List cacheData = Lists.newLinkedList(); - return Sequences.simple( - Iterables.transform( - segmentResults, - toolChest.makeMetricManipulatorFn( + return Sequences.withEffect( + Sequences.map( + Sequences.map( + Sequences.simple(value.getResults()), + new Function() + { + @Override + public T apply(T input) + { + cacheData.add(cacheFn.apply(input)); + return input; + } + } + ), + toolChest.makePreComputeManipulatorFn( rewrittenQuery, new MetricManipulationFn() { @@ -364,7 +368,21 @@ public class CachingClusteredClient implements QueryRunner } } ) - ) + ), + new Runnable() + { + @Override + public void run() + { + CachePopulator cachePopulator = cachePopulatorMap.get( + String.format("%s_%s", value.getSegmentId(), value.getInterval()) + ); + if (cachePopulator != null) { + cachePopulator.populate(cacheData); + } + } + }, + MoreExecutors.sameThreadExecutor() ); } } diff --git a/server/src/main/java/io/druid/client/CachingQueryRunner.java b/server/src/main/java/io/druid/client/CachingQueryRunner.java new file mode 100644 index 00000000000..ad2718e6a45 --- /dev/null +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -0,0 +1,169 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 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.client; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.guava.BaseSequence; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; +import io.druid.query.CacheStrategy; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.SegmentDescriptor; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; + +public class CachingQueryRunner implements QueryRunner +{ + + private final String segmentIdentifier; + private final SegmentDescriptor segmentDescriptor; + private final QueryRunner base; + private final QueryToolChest toolChest; + private final Cache cache; + private final ObjectMapper mapper; + private final CacheConfig cacheConfig; + + public CachingQueryRunner( + String segmentIdentifier, + SegmentDescriptor segmentDescriptor, + ObjectMapper mapper, + Cache cache, + QueryToolChest toolchest, + QueryRunner base, + CacheConfig cacheConfig + ) + { + this.base = base; + this.segmentIdentifier = segmentIdentifier; + this.segmentDescriptor = segmentDescriptor; + this.toolChest = toolchest; + this.cache = cache; + this.mapper = mapper; + this.cacheConfig = cacheConfig; + } + + @Override + public Sequence run(Query query) + { + final CacheStrategy strategy = toolChest.getCacheStrategy(query); + + final boolean populateCache = query.getContextPopulateCache(true) + && strategy != null + && cacheConfig.isPopulateCache(); + + final boolean useCache = query.getContextUseCache(true) + && strategy != null + && cacheConfig.isUseCache(); + + final Cache.NamedKey key; + if(strategy != null && (useCache || populateCache)) { + key = CacheUtil.computeSegmentCacheKey( + segmentIdentifier, + segmentDescriptor, + strategy.computeCacheKey(query) + ); + } else { + key = null; + } + + if(useCache) { + final Function cacheFn = strategy.pullFromCache(); + final byte[] cachedResult = cache.get(key); + if(cachedResult != null) { + final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); + + return Sequences.map( + new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + try { + if (cachedResult.length == 0) { + return Iterators.emptyIterator(); + } + + return mapper.readValues( + mapper.getFactory().createParser(cachedResult), + cacheObjectClazz + ); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + @Override + public void cleanup(Iterator iterFromMake) + { + } + } + ), + cacheFn + ); + } + } + + if (populateCache) { + final Function cacheFn = strategy.prepareForCache(); + final List cacheResults = Lists.newLinkedList(); + + return Sequences.withEffect( + Sequences.map( + base.run(query), + new Function() + { + @Override + public T apply(T input) + { + cacheResults.add(cacheFn.apply(input)); + return input; + } + } + ), + new Runnable() + { + @Override + public void run() + { + CacheUtil.populate(cache, mapper, key, cacheResults); + } + }, + MoreExecutors.sameThreadExecutor() + ); + } else { + return base.run(query); + } + } + +} diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index fa95ba97f11..1194acf8ce1 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -106,7 +106,7 @@ public class DirectDruidClient implements QueryRunner public Sequence run(Query query) { QueryToolChest> toolChest = warehouse.getToolChest(query); - boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment", "false")); + boolean isBySegment = query.getContextBySegment(false); Pair types = typesMap.get(query.getClass()); if (types == null) { @@ -122,8 +122,7 @@ public class DirectDruidClient implements QueryRunner final JavaType typeRef; if (isBySegment) { typeRef = types.rhs; - } - else { + } else { typeRef = types.lhs; } @@ -218,15 +217,16 @@ public class DirectDruidClient implements QueryRunner if (!isBySegment) { retVal = Sequences.map( retVal, - toolChest.makeMetricManipulatorFn( - query, new MetricManipulationFn() - { - @Override - public Object manipulate(AggregatorFactory factory, Object object) - { - return factory.deserialize(object); - } - } + toolChest.makePreComputeManipulatorFn( + query, + new MetricManipulationFn() + { + @Override + public Object manipulate(AggregatorFactory factory, Object object) + { + return factory.deserialize(object); + } + } ) ); } @@ -313,7 +313,7 @@ public class DirectDruidClient implements QueryRunner @Override public void close() throws IOException { - if(jp != null) { + if (jp != null) { jp.close(); } } diff --git a/server/src/main/java/io/druid/client/RoutingDruidClient.java b/server/src/main/java/io/druid/client/RoutingDruidClient.java index 9fd3e2b0eac..10170fcfb9e 100644 --- a/server/src/main/java/io/druid/client/RoutingDruidClient.java +++ b/server/src/main/java/io/druid/client/RoutingDruidClient.java @@ -28,8 +28,9 @@ import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.logger.Logger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.HttpResponseHandler; -import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Client; import io.druid.query.Query; +import io.druid.server.router.Router; import org.jboss.netty.handler.codec.http.HttpHeaders; import javax.inject.Inject; @@ -52,7 +53,7 @@ public class RoutingDruidClient @Inject public RoutingDruidClient( ObjectMapper objectMapper, - @Global HttpClient httpClient + @Router HttpClient httpClient ) { this.objectMapper = objectMapper; @@ -67,14 +68,13 @@ public class RoutingDruidClient return openConnections.get(); } - public ListenableFuture run( - String host, + public ListenableFuture post( + String url, Query query, HttpResponseHandler responseHandler ) { final ListenableFuture future; - final String url = String.format("http://%s/druid/v2/", host); try { log.debug("Querying url[%s]", url); @@ -110,4 +110,19 @@ public class RoutingDruidClient return future; } + + public ListenableFuture get( + String url, + HttpResponseHandler responseHandler + ) + { + try { + return httpClient + .get(new URL(url)) + .go(responseHandler); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } } diff --git a/server/src/main/java/io/druid/client/cache/LocalCacheProvider.java b/server/src/main/java/io/druid/client/cache/LocalCacheProvider.java index 716f5abab93..62cd3d9caaa 100644 --- a/server/src/main/java/io/druid/client/cache/LocalCacheProvider.java +++ b/server/src/main/java/io/druid/client/cache/LocalCacheProvider.java @@ -29,7 +29,7 @@ public class LocalCacheProvider implements CacheProvider { @JsonProperty @Min(0) - private long sizeInBytes = 10485760; + private long sizeInBytes = 0; @JsonProperty @Min(0) diff --git a/server/src/main/java/io/druid/client/cache/MapCache.java b/server/src/main/java/io/druid/client/cache/MapCache.java index fd03fac61d7..5221fd0ca91 100644 --- a/server/src/main/java/io/druid/client/cache/MapCache.java +++ b/server/src/main/java/io/druid/client/cache/MapCache.java @@ -77,7 +77,10 @@ public class MapCache implements Cache @Override public byte[] get(NamedKey key) { - final byte[] retVal = baseMap.get(computeKey(getNamespaceId(key.namespace), key.key)); + final byte[] retVal; + synchronized (clearLock) { + retVal = baseMap.get(computeKey(getNamespaceId(key.namespace), key.key)); + } if (retVal == null) { missCount.incrementAndGet(); } else { @@ -90,7 +93,7 @@ public class MapCache implements Cache public void put(NamedKey key, byte[] value) { synchronized (clearLock) { - baseMap.put(computeKey(getNamespaceId(key.namespace), key.key), value); + baseMap.put(computeKey(getNamespaceId(key.namespace), key.key), value); } } @@ -98,7 +101,7 @@ public class MapCache implements Cache public Map getBulk(Iterable keys) { Map retVal = Maps.newHashMap(); - for(NamedKey key : keys) { + for (NamedKey key : keys) { retVal.put(key, get(key)); } return retVal; @@ -110,7 +113,9 @@ public class MapCache implements Cache byte[] idBytes; synchronized (namespaceId) { idBytes = getNamespaceId(namespace); - if(idBytes == null) return; + if (idBytes == null) { + return; + } namespaceId.remove(namespace); } @@ -150,7 +155,8 @@ public class MapCache implements Cache return retVal; } - public boolean isLocal() { + public boolean isLocal() + { return true; } } diff --git a/server/src/main/java/io/druid/curator/announcement/Announcer.java b/server/src/main/java/io/druid/curator/announcement/Announcer.java index f9ff444bf5e..99393a7af34 100644 --- a/server/src/main/java/io/druid/curator/announcement/Announcer.java +++ b/server/src/main/java/io/druid/curator/announcement/Announcer.java @@ -212,11 +212,6 @@ public class Announcer pathsToReinstate.add(path); } - for (ChildData data : cache.getCurrentData()) { - log.info("Node[%s] in cache is removed to reinstate.", data.getPath()); - pathsToReinstate.remove(data.getPath()); - } - if (!pathsToReinstate.isEmpty() && !pathsLost.compareAndSet(null, pathsToReinstate)) { log.info("Already had a pathsLost set!?[%s]", parentPath); } diff --git a/server/src/main/java/io/druid/segment/realtime/DbSegmentPublisher.java b/server/src/main/java/io/druid/segment/realtime/DbSegmentPublisher.java index b97738aa91a..bef339a9c2c 100644 --- a/server/src/main/java/io/druid/segment/realtime/DbSegmentPublisher.java +++ b/server/src/main/java/io/druid/segment/realtime/DbSegmentPublisher.java @@ -25,6 +25,7 @@ import com.metamx.common.logger.Logger; import io.druid.db.DbConnector; import io.druid.db.DbTablesConfig; import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NoneShardSpec; import org.joda.time.DateTime; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.IDBI; @@ -104,7 +105,7 @@ public class DbSegmentPublisher implements SegmentPublisher .bind("created_date", new DateTime().toString()) .bind("start", segment.getInterval().getStart().toString()) .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", segment.getShardSpec().getPartitionNum()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? 0 : 1) .bind("version", segment.getVersion()) .bind("used", true) .bind("payload", jsonMapper.writeValueAsString(segment)) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 80bd59235c5..0210219a3d3 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -68,7 +68,6 @@ import java.util.concurrent.ScheduledExecutorService; public class RealtimePlumber implements Plumber { private static final EmittingLogger log = new EmittingLogger(RealtimePlumber.class); - private final Period windowPeriod; private final File basePersistDirectory; private final IndexGranularity segmentGranularity; @@ -84,16 +83,15 @@ public class RealtimePlumber implements Plumber private final SegmentPublisher segmentPublisher; private final ServerView serverView; private final int maxPendingPersists; - private final Object handoffCondition = new Object(); private final Map sinks = Maps.newConcurrentMap(); private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline( String.CASE_INSENSITIVE_ORDER ); - private volatile boolean shuttingDown = false; private volatile boolean stopped = false; private volatile ExecutorService persistExecutor = null; + private volatile ExecutorService mergeExecutor = null; private volatile ScheduledExecutorService scheduledExecutor = null; public RealtimePlumber( @@ -300,13 +298,13 @@ public class RealtimePlumber implements Plumber ); } - // Submits persist-n-merge task for a Sink to the persistExecutor + // Submits persist-n-merge task for a Sink to the mergeExecutor private void persistAndMerge(final long truncatedTime, final Sink sink) { final String threadName = String.format( "%s-%s-persist-n-merge", schema.getDataSource(), new DateTime(truncatedTime) ); - persistExecutor.execute( + mergeExecutor.execute( new ThreadRenamingRunnable(threadName) { @Override @@ -431,6 +429,13 @@ public class RealtimePlumber implements Plumber "plumber_persist_%d", maxPendingPersists ); } + if (mergeExecutor == null) { + // use a blocking single threaded executor to throttle the firehose when write to disk is slow + mergeExecutor = Execs.newBlockingSingleThreaded( + "plumber_merge_%d", 1 + ); + } + if (scheduledExecutor == null) { scheduledExecutor = Executors.newScheduledThreadPool( 1, @@ -592,7 +597,11 @@ public class RealtimePlumber implements Plumber log.info("Adding entry[%s] for merge and push.", entry); sinksToPush.add(entry); } else { - log.warn("[%s] < [%s] Skipping persist and merge.", new DateTime(intervalStart), minTimestampAsDate); + log.warn( + "[%s] < [%s] Skipping persist and merge.", + new DateTime(intervalStart), + minTimestampAsDate + ); } } @@ -660,39 +669,46 @@ public class RealtimePlumber implements Plumber */ protected int persistHydrant(FireHydrant indexToPersist, Schema schema, Interval interval) { - if (indexToPersist.hasSwapped()) { + synchronized (indexToPersist) { + if (indexToPersist.hasSwapped()) { + log.info( + "DataSource[%s], Interval[%s], Hydrant[%s] already swapped. Ignoring request to persist.", + schema.getDataSource(), interval, indexToPersist + ); + return 0; + } + log.info( - "DataSource[%s], Interval[%s], Hydrant[%s] already swapped. Ignoring request to persist.", - schema.getDataSource(), interval, indexToPersist + "DataSource[%s], Interval[%s], persisting Hydrant[%s]", + schema.getDataSource(), + interval, + indexToPersist ); - return 0; - } + try { + int numRows = indexToPersist.getIndex().size(); - log.info("DataSource[%s], Interval[%s], persisting Hydrant[%s]", schema.getDataSource(), interval, indexToPersist); - try { - int numRows = indexToPersist.getIndex().size(); + File persistedFile = IndexMerger.persist( + indexToPersist.getIndex(), + new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) + ); - File persistedFile = IndexMerger.persist( - indexToPersist.getIndex(), - new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) - ); + indexToPersist.swapSegment( + new QueryableIndexSegment( + indexToPersist.getSegment().getIdentifier(), + IndexIO.loadIndex(persistedFile) + ) + ); - indexToPersist.swapSegment( - new QueryableIndexSegment( - indexToPersist.getSegment().getIdentifier(), - IndexIO.loadIndex(persistedFile) - ) - ); + return numRows; + } + catch (IOException e) { + log.makeAlert("dataSource[%s] -- incremental persist failed", schema.getDataSource()) + .addData("interval", interval) + .addData("count", indexToPersist.getCount()) + .emit(); - return numRows; - } - catch (IOException e) { - log.makeAlert("dataSource[%s] -- incremental persist failed", schema.getDataSource()) - .addData("interval", interval) - .addData("count", indexToPersist.getCount()) - .emit(); - - throw Throwables.propagate(e); + throw Throwables.propagate(e); + } } } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index 4a8332137d4..447caa4a7e6 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -117,6 +117,12 @@ public class RealtimePlumberSchool implements PlumberSchool this.rejectionPolicyFactory = factory; } + @JsonProperty("maxPendingPersists") + public void setDefaultMaxPendingPersists(int maxPendingPersists) + { + this.maxPendingPersists = maxPendingPersists; + } + public void setEmitter(ServiceEmitter emitter) { this.emitter = emitter; @@ -152,11 +158,6 @@ public class RealtimePlumberSchool implements PlumberSchool this.queryExecutorService = executorService; } - public void setDefaultMaxPendingPersists(int maxPendingPersists) - { - this.maxPendingPersists = maxPendingPersists; - } - @Override public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics) { diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 85f33a70007..7147ab9e87c 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -21,6 +21,8 @@ package io.druid.server; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableMap; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; @@ -46,8 +48,10 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.io.OutputStream; import java.nio.charset.Charset; +import java.util.UUID; /** + * This class does async query processing and should be merged with QueryResource at some point */ @WebServlet(asyncSupported = true) public class AsyncQueryForwardingServlet extends HttpServlet @@ -55,6 +59,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet private static final EmittingLogger log = new EmittingLogger(AsyncQueryForwardingServlet.class); private static final Charset UTF8 = Charset.forName("UTF-8"); private static final String DISPATCHED = "dispatched"; + private static final Joiner COMMA_JOIN = Joiner.on(","); private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; @@ -62,7 +67,6 @@ public class AsyncQueryForwardingServlet extends HttpServlet private final RoutingDruidClient routingDruidClient; private final ServiceEmitter emitter; private final RequestLogger requestLogger; - private final QueryIDProvider idProvider; public AsyncQueryForwardingServlet( @Json ObjectMapper jsonMapper, @@ -70,8 +74,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet QueryHostFinder hostFinder, RoutingDruidClient routingDruidClient, ServiceEmitter emitter, - RequestLogger requestLogger, - QueryIDProvider idProvider + RequestLogger requestLogger ) { this.jsonMapper = jsonMapper; @@ -80,7 +83,116 @@ public class AsyncQueryForwardingServlet extends HttpServlet this.routingDruidClient = routingDruidClient; this.emitter = emitter; this.requestLogger = requestLogger; - this.idProvider = idProvider; + } + + @Override + protected void doGet(final HttpServletRequest req, final HttpServletResponse resp) + throws ServletException, IOException + { + OutputStream out = null; + AsyncContext ctx = null; + + try { + ctx = req.startAsync(req, resp); + final AsyncContext asyncContext = ctx; + + if (req.getAttribute(DISPATCHED) != null) { + return; + } + + out = resp.getOutputStream(); + final OutputStream outputStream = out; + + final String host = hostFinder.getDefaultHost(); + + final HttpResponseHandler responseHandler = new HttpResponseHandler() + { + @Override + public ClientResponse handleResponse(HttpResponse response) + { + resp.setStatus(response.getStatus().getCode()); + resp.setContentType("application/json"); + + try { + ChannelBuffer buf = response.getContent(); + buf.readBytes(outputStream, buf.readableBytes()); + } + catch (Exception e) { + asyncContext.complete(); + throw Throwables.propagate(e); + } + + return ClientResponse.finished(outputStream); + } + + @Override + public ClientResponse handleChunk( + ClientResponse clientResponse, HttpChunk chunk + ) + { + try { + ChannelBuffer buf = chunk.getContent(); + buf.readBytes(outputStream, buf.readableBytes()); + } + catch (Exception e) { + asyncContext.complete(); + throw Throwables.propagate(e); + } + return clientResponse; + } + + @Override + public ClientResponse done(ClientResponse clientResponse) + { + final OutputStream obj = clientResponse.getObj(); + try { + resp.flushBuffer(); + outputStream.close(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + finally { + asyncContext.complete(); + } + + return ClientResponse.finished(obj); + } + }; + + asyncContext.start( + new Runnable() + { + @Override + public void run() + { + routingDruidClient.get(makeUrl(host, req), responseHandler); + } + } + ); + + asyncContext.dispatch(); + req.setAttribute(DISPATCHED, true); + } + catch (Exception e) { + if (!resp.isCommitted()) { + resp.setStatus(500); + resp.resetBuffer(); + + if (out == null) { + out = resp.getOutputStream(); + } + + if (ctx != null) { + ctx.complete(); + } + + out.write((e.getMessage() == null) ? "Exception null".getBytes(UTF8) : e.getMessage().getBytes(UTF8)); + out.write("\n".getBytes(UTF8)); + } + + resp.flushBuffer(); + } } @Override @@ -94,31 +206,30 @@ public class AsyncQueryForwardingServlet extends HttpServlet final boolean isSmile = "application/smile".equals(req.getContentType()); - ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; + final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; OutputStream out = null; + AsyncContext ctx = null; try { - final AsyncContext ctx = req.startAsync(req, resp); + ctx = req.startAsync(req, resp); + final AsyncContext asyncContext = ctx; if (req.getAttribute(DISPATCHED) != null) { return; } - req.setAttribute(DISPATCHED, true); - resp.setStatus(200); - resp.setContentType("application/x-javascript"); - query = objectMapper.readValue(req.getInputStream(), Query.class); queryId = query.getId(); if (queryId == null) { - queryId = idProvider.next(query); + queryId = UUID.randomUUID().toString(); query = query.withId(queryId); } - requestLogger.log( - new RequestLogLine(new DateTime(), req.getRemoteAddr(), query) - ); + if (log.isDebugEnabled()) { + log.debug("Got query [%s]", query); + } + out = resp.getOutputStream(); final OutputStream outputStream = out; @@ -132,14 +243,16 @@ public class AsyncQueryForwardingServlet extends HttpServlet @Override public ClientResponse handleResponse(HttpResponse response) { - byte[] bytes = getContentBytes(response.getContent()); - if (bytes.length > 0) { - try { - outputStream.write(bytes); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + resp.setStatus(response.getStatus().getCode()); + resp.setContentType("application/x-javascript"); + + try { + ChannelBuffer buf = response.getContent(); + buf.readBytes(outputStream, buf.readableBytes()); + } + catch (Exception e) { + asyncContext.complete(); + throw Throwables.propagate(e); } return ClientResponse.finished(outputStream); } @@ -149,14 +262,13 @@ public class AsyncQueryForwardingServlet extends HttpServlet ClientResponse clientResponse, HttpChunk chunk ) { - byte[] bytes = getContentBytes(chunk.getContent()); - if (bytes.length > 0) { - try { - clientResponse.getObj().write(bytes); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + try { + ChannelBuffer buf = chunk.getContent(); + buf.readBytes(outputStream, buf.readableBytes()); + } + catch (Exception e) { + asyncContext.complete(); + throw Throwables.propagate(e); } return clientResponse; } @@ -166,13 +278,13 @@ public class AsyncQueryForwardingServlet extends HttpServlet { final long requestTime = System.currentTimeMillis() - start; - log.info("Request time: %d", requestTime); + log.debug("Request time: %d", requestTime); emitter.emit( new ServiceMetricEvent.Builder() .setUser2(theQuery.getDataSource().getName()) .setUser4(theQuery.getType()) - .setUser5(theQuery.getIntervals().get(0).toString()) + .setUser5(COMMA_JOIN.join(theQuery.getIntervals())) .setUser6(String.valueOf(theQuery.hasFilters())) .setUser7(req.getRemoteAddr()) .setUser8(theQueryId) @@ -182,6 +294,15 @@ public class AsyncQueryForwardingServlet extends HttpServlet final OutputStream obj = clientResponse.getObj(); try { + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + theQuery, + new QueryStats(ImmutableMap.of("request/time", requestTime, "success", true)) + ) + ); + resp.flushBuffer(); outputStream.close(); } @@ -189,30 +310,26 @@ public class AsyncQueryForwardingServlet extends HttpServlet throw Throwables.propagate(e); } finally { - ctx.dispatch(); + asyncContext.complete(); } return ClientResponse.finished(obj); } - - private byte[] getContentBytes(ChannelBuffer content) - { - byte[] contentBytes = new byte[content.readableBytes()]; - content.readBytes(contentBytes); - return contentBytes; - } }; - ctx.start( + asyncContext.start( new Runnable() { @Override public void run() { - routingDruidClient.run(host, theQuery, responseHandler); + routingDruidClient.post(makeUrl(host, req), theQuery, responseHandler); } } ); + + asyncContext.dispatch(); + req.setAttribute(DISPATCHED, true); } catch (Exception e) { if (!resp.isCommitted()) { @@ -229,10 +346,39 @@ public class AsyncQueryForwardingServlet extends HttpServlet resp.flushBuffer(); + if (ctx != null) { + ctx.complete(); + } + + try { + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + query, + new QueryStats(ImmutableMap.of("success", false, "exception", e.toString())) + ) + ); + } + catch (Exception e2) { + log.error(e2, "Unable to log query [%s]!", query); + } + log.makeAlert(e, "Exception handling request") .addData("query", query) .addData("peer", req.getRemoteAddr()) .emit(); } } + + private String makeUrl(final String host, final HttpServletRequest req) + { + final String queryString = req.getQueryString(); + final String requestURI = req.getRequestURI() == null ? "" : req.getRequestURI(); + + if (queryString == null) { + return String.format("http://%s%s", host, requestURI); + } + return String.format("http://%s%s?%s", host, requestURI, queryString); + } } diff --git a/server/src/main/java/io/druid/server/ClientInfoResource.java b/server/src/main/java/io/druid/server/ClientInfoResource.java index 782029f29b5..cbd253525bb 100644 --- a/server/src/main/java/io/druid/server/ClientInfoResource.java +++ b/server/src/main/java/io/druid/server/ClientInfoResource.java @@ -101,7 +101,12 @@ public class ClientInfoResource @QueryParam("interval") String interval ) { - List segments = getSegmentsForDatasources().get(dataSourceName); + final List segments = getSegmentsForDatasources().get(dataSourceName); + final Set dims = Sets.newHashSet(); + + if (segments == null || segments.isEmpty()) { + return dims; + } Interval theInterval; if (interval == null || interval.isEmpty()) { @@ -111,7 +116,6 @@ public class ClientInfoResource theInterval = new Interval(interval); } - Set dims = Sets.newHashSet(); for (DataSegment segment : segments) { if (theInterval.overlaps(segment.getInterval())) { dims.addAll(segment.getDimensions()); @@ -129,8 +133,13 @@ public class ClientInfoResource @QueryParam("interval") String interval ) { - List segments = getSegmentsForDatasources().get(dataSourceName); + final List segments = getSegmentsForDatasources().get(dataSourceName); + final Set metrics = Sets.newHashSet(); + if (segments == null || segments.isEmpty()) { + return metrics; + } + Interval theInterval; if (interval == null || interval.isEmpty()) { DateTime now = new DateTime(); @@ -139,7 +148,6 @@ public class ClientInfoResource theInterval = new Interval(interval); } - Set metrics = Sets.newHashSet(); for (DataSegment segment : segments) { if (theInterval.overlaps(segment.getInterval())) { metrics.addAll(segment.getMetrics()); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index a69a3859503..77d0c1bf04c 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -22,14 +22,14 @@ package io.druid.server; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Charsets; +import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; import com.google.inject.Inject; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.common.logger.Logger; -import com.metamx.emitter.service.AlertEvent; +import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.guice.annotations.Json; @@ -49,20 +49,22 @@ import javax.ws.rs.core.Context; import java.io.IOException; import java.io.OutputStream; import java.nio.charset.Charset; +import java.util.UUID; /** */ @Path("/druid/v2/") public class QueryResource { - private static final Logger log = new Logger(QueryResource.class); + private static final EmittingLogger log = new EmittingLogger(QueryResource.class); private static final Charset UTF8 = Charset.forName("UTF-8"); + private static final Joiner COMMA_JOIN = Joiner.on(","); + private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; private final QuerySegmentWalker texasRanger; private final ServiceEmitter emitter; private final RequestLogger requestLogger; - private final QueryIDProvider idProvider; @Inject public QueryResource( @@ -70,8 +72,7 @@ public class QueryResource @Smile ObjectMapper smileMapper, QuerySegmentWalker texasRanger, ServiceEmitter emitter, - RequestLogger requestLogger, - QueryIDProvider idProvider + RequestLogger requestLogger ) { this.jsonMapper = jsonMapper; @@ -79,7 +80,6 @@ public class QueryResource this.texasRanger = texasRanger; this.emitter = emitter; this.requestLogger = requestLogger; - this.idProvider = idProvider; } @POST @@ -107,13 +107,13 @@ public class QueryResource query = objectMapper.readValue(requestQuery, Query.class); queryId = query.getId(); if (queryId == null) { - queryId = idProvider.next(query); + queryId = UUID.randomUUID().toString(); query = query.withId(queryId); } - requestLogger.log( - new RequestLogLine(new DateTime(), req.getRemoteAddr(), query) - ); + if (log.isDebugEnabled()) { + log.debug("Got query [%s]", query); + } Sequence results = query.run(texasRanger); @@ -133,13 +133,27 @@ public class QueryResource new ServiceMetricEvent.Builder() .setUser2(query.getDataSource().toString()) .setUser4(query.getType()) - .setUser5(query.getIntervals().get(0).toString()) + .setUser5(COMMA_JOIN.join(query.getIntervals())) .setUser6(String.valueOf(query.hasFilters())) .setUser7(req.getRemoteAddr()) .setUser8(queryId) .setUser9(query.getDuration().toPeriod().toStandardMinutes().toString()) .build("request/time", requestTime) ); + + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + query, + new QueryStats( + ImmutableMap.of( + "request/time", requestTime, + "success", true + ) + ) + ) + ); } catch (Exception e) { final String queryString = @@ -163,16 +177,25 @@ public class QueryResource resp.flushBuffer(); - emitter.emit( - new AlertEvent.Builder().build( - "Exception handling request", - ImmutableMap.builder() - .put("exception", e.toString()) - .put("query", queryString) - .put("peer", req.getRemoteAddr()) - .build() - ) - ); + try { + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + query, + new QueryStats(ImmutableMap.of("success", false, "exception", e.toString())) + ) + ); + } + catch (Exception e2) { + log.error(e2, "Unable to log query [%s]!", queryString); + } + + log.makeAlert(e, "Exception handling request") + .addData("exception", e.toString()) + .addData("query", queryString) + .addData("peer", req.getRemoteAddr()) + .emit(); } finally { resp.flushBuffer(); diff --git a/server/src/main/java/io/druid/server/QueryIDProvider.java b/server/src/main/java/io/druid/server/QueryStats.java similarity index 54% rename from server/src/main/java/io/druid/server/QueryIDProvider.java rename to server/src/main/java/io/druid/server/QueryStats.java index 9226af1606b..638dbb09b7a 100644 --- a/server/src/main/java/io/druid/server/QueryIDProvider.java +++ b/server/src/main/java/io/druid/server/QueryStats.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * 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 @@ -19,35 +19,24 @@ package io.druid.server; -import com.google.inject.Inject; -import com.google.inject.Singleton; -import io.druid.guice.annotations.Self; -import io.druid.query.Query; -import org.joda.time.DateTime; +import com.fasterxml.jackson.annotation.JsonValue; -import java.util.concurrent.atomic.AtomicLong; +import java.util.Map; -@Singleton -public class QueryIDProvider +/** + */ +public class QueryStats { - private final String host; - private final AtomicLong id = new AtomicLong(); + private final Map stats; - @Inject - public QueryIDProvider(@Self DruidNode node) + public QueryStats(Map stats) { - host = node.getHost(); + this.stats = stats; } - public String next(Query query) + @JsonValue + public Map getStats() { - return String.format( - "%s_%s_%s_%s_%s", - query.getDataSource(), - query.getDuration(), - host, - new DateTime(), - id.incrementAndGet() - ); + return stats; } } diff --git a/server/src/main/java/io/druid/server/RequestLogLine.java b/server/src/main/java/io/druid/server/RequestLogLine.java index a52485ae279..6141bb44319 100644 --- a/server/src/main/java/io/druid/server/RequestLogLine.java +++ b/server/src/main/java/io/druid/server/RequestLogLine.java @@ -35,12 +35,14 @@ public class RequestLogLine private final DateTime timestamp; private final String remoteAddr; private final Query query; + private final QueryStats queryStats; - public RequestLogLine(DateTime timestamp, String remoteAddr, Query query) + public RequestLogLine(DateTime timestamp, String remoteAddr, Query query, QueryStats queryStats) { this.timestamp = timestamp; this.remoteAddr = remoteAddr; this.query = query; + this.queryStats = queryStats; } public String getLine(ObjectMapper objectMapper) throws JsonProcessingException @@ -49,7 +51,8 @@ public class RequestLogLine Arrays.asList( timestamp, remoteAddr, - objectMapper.writeValueAsString(query) + objectMapper.writeValueAsString(query), + objectMapper.writeValueAsString(queryStats) ) ); } @@ -71,4 +74,10 @@ public class RequestLogLine { return remoteAddr; } + + @JsonProperty("queryStats") + public QueryStats getQueryStats() + { + return queryStats; + } } diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index b49fade0404..2869237a8c7 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -29,7 +29,7 @@ import com.metamx.common.guava.FunctionalIterable; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import io.druid.client.CachePopulatingQueryRunner; +import io.druid.client.CachingQueryRunner; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; import io.druid.collections.CountingMap; @@ -410,7 +410,7 @@ public class ServerManager implements QuerySegmentWalker new BySegmentQueryRunner( adapter.getIdentifier(), adapter.getDataInterval().getStart(), - new CachePopulatingQueryRunner( + new CachingQueryRunner( adapter.getIdentifier(), segmentDescriptor, objectMapper, @@ -424,4 +424,4 @@ public class ServerManager implements QuerySegmentWalker segmentSpec ); } -} \ No newline at end of file +} diff --git a/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java index 70df84450a6..5a71415901d 100644 --- a/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java @@ -26,6 +26,7 @@ import com.metamx.emitter.core.Event; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; import io.druid.query.Query; +import io.druid.server.QueryStats; import io.druid.server.RequestLogLine; import org.joda.time.DateTime; @@ -110,6 +111,12 @@ public class EmittingRequestLogger implements RequestLogger return request.getRemoteAddr(); } + @JsonProperty("queryStats") + public QueryStats getQueryStats() + { + return request.getQueryStats(); + } + @Override public boolean isSafeToBuffer() { diff --git a/server/src/main/java/io/druid/server/metrics/ServerMonitor.java b/server/src/main/java/io/druid/server/metrics/ServerMonitor.java index 5005d0ac50f..5379dcd2c8e 100644 --- a/server/src/main/java/io/druid/server/metrics/ServerMonitor.java +++ b/server/src/main/java/io/druid/server/metrics/ServerMonitor.java @@ -59,7 +59,8 @@ public class ServerMonitor extends AbstractMonitor .setUser2(serverConfig.getTier()); emitter.emit(builder.build("server/segment/used", used)); - emitter.emit(builder.build("server/segment/usedPercent", used / (double) serverConfig.getMaxSize())); + final double usedPercent = serverConfig.getMaxSize() == 0 ? 0 : used / (double) serverConfig.getMaxSize(); + emitter.emit(builder.build("server/segment/usedPercent", usedPercent)); } for (Map.Entry entry : serverManager.getDataSourceCounts().entrySet()) { @@ -74,7 +75,8 @@ public class ServerMonitor extends AbstractMonitor final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder().setUser2(serverConfig.getTier()); emitter.emit(builder.build("server/segment/totalUsed", totalUsed)); - emitter.emit(builder.build("server/segment/totalUsedPercent", totalUsed / (double) serverConfig.getMaxSize())); + final double totalUsedPercent = serverConfig.getMaxSize() == 0 ? 0 : totalUsed / (double) serverConfig.getMaxSize(); + emitter.emit(builder.build("server/segment/totalUsedPercent", totalUsedPercent)); emitter.emit(builder.build("server/segment/totalCount", totalCount)); return true; diff --git a/server/src/main/java/io/druid/server/router/QueryHostFinder.java b/server/src/main/java/io/druid/server/router/QueryHostFinder.java index 3355f77b22b..ed2c8096c5a 100644 --- a/server/src/main/java/io/druid/server/router/QueryHostFinder.java +++ b/server/src/main/java/io/druid/server/router/QueryHostFinder.java @@ -20,6 +20,7 @@ package io.druid.server.router; import com.google.inject.Inject; +import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.emitter.EmittingLogger; import io.druid.client.selector.Server; @@ -49,11 +50,57 @@ public class QueryHostFinder public Server findServer(Query query) { final Pair selected = hostSelector.select(query); + return findServerInner(selected); + } - final String serviceName = selected.lhs; - final ServerDiscoverySelector selector = selected.rhs; + public Server findDefaultServer() + { + final Pair selected = hostSelector.getDefaultLookup(); + return findServerInner(selected); + } - Server server = selector.pick(); + public String getHost(Query query) + { + Server server = findServer(query); + + if (server == null) { + log.makeAlert( + "Catastrophic failure! No servers found at all! Failing request!" + ).emit(); + + throw new ISE("No server found for query[%s]", query); + } + + log.debug("Selected [%s]", server.getHost()); + + return server.getHost(); + } + + public String getDefaultHost() + { + Server server = findDefaultServer(); + + if (server == null) { + log.makeAlert( + "Catastrophic failure! No servers found at all! Failing request!" + ).emit(); + + throw new ISE("No default server found!"); + } + + return server.getHost(); + } + + private Server findServerInner(final Pair selected) + { + if (selected == null) { + log.error("Danger, Will Robinson! Unable to find any brokers!"); + } + + final String serviceName = selected == null ? hostSelector.getDefaultServiceName() : selected.lhs; + final ServerDiscoverySelector selector = selected == null ? null : selected.rhs; + + Server server = selector == null ? null : selector.pick(); if (server == null) { log.error( "WTF?! No server found for serviceName[%s]. Using backup", @@ -78,21 +125,4 @@ public class QueryHostFinder return server; } - - public String getHost(Query query) - { - Server server = findServer(query); - - if (server == null) { - log.makeAlert( - "Catastrophic failure! No servers found at all! Failing request!" - ).emit(); - - return null; - } - - log.debug("Selected [%s]", server.getHost()); - - return server.getHost(); - } } diff --git a/server/src/main/java/io/druid/server/router/Router.java b/server/src/main/java/io/druid/server/router/Router.java new file mode 100644 index 00000000000..0677c80691e --- /dev/null +++ b/server/src/main/java/io/druid/server/router/Router.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.router; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Router +{ +} diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java index cc625cdc5a3..681acd815b1 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java @@ -122,7 +122,7 @@ public class TieredBrokerHostSelector implements HostSelector { synchronized (lock) { if (!ruleManager.isStarted() || !started) { - return null; + return getDefaultLookup(); } } @@ -157,7 +157,7 @@ public class TieredBrokerHostSelector implements HostSelector } if (baseRule == null) { - return null; + return getDefaultLookup(); } // in the baseRule, find the broker of highest priority @@ -170,26 +170,33 @@ public class TieredBrokerHostSelector implements HostSelector } if (brokerServiceName == null) { - log.makeAlert( + log.error( "WTF?! No brokerServiceName found for datasource[%s], intervals[%s]. Using default[%s].", query.getDataSource(), query.getIntervals(), tierConfig.getDefaultBrokerServiceName() - ).emit(); + ); brokerServiceName = tierConfig.getDefaultBrokerServiceName(); } ServerDiscoverySelector retVal = selectorMap.get(brokerServiceName); if (retVal == null) { - log.makeAlert( + log.error( "WTF?! No selector found for brokerServiceName[%s]. Using default selector for[%s]", brokerServiceName, tierConfig.getDefaultBrokerServiceName() - ).emit(); + ); retVal = selectorMap.get(tierConfig.getDefaultBrokerServiceName()); } return new Pair<>(brokerServiceName, retVal); } + + public Pair getDefaultLookup() + { + final String brokerServiceName = tierConfig.getDefaultBrokerServiceName(); + final ServerDiscoverySelector retVal = selectorMap.get(brokerServiceName); + return new Pair<>(brokerServiceName, retVal); + } } diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java index 5f9635cd650..5110f886601 100644 --- a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -51,10 +51,10 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec @Override public boolean isInChunk(InputRow inputRow) { - return Math.abs(hash(inputRow)) % getPartitions() == getPartitionNum(); + return (((long) hash(inputRow)) - getPartitionNum()) % getPartitions() == 0; } - private int hash(InputRow inputRow) + protected int hash(InputRow inputRow) { final List groupKey = Rows.toGroupKey(inputRow.getTimestampFromEpoch(), inputRow); try { diff --git a/server/src/test/java/io/druid/client/CachePopulatingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachePopulatingQueryRunnerTest.java deleted file mode 100644 index b87cb15b317..00000000000 --- a/server/src/test/java/io/druid/client/CachePopulatingQueryRunnerTest.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * 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.client; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.metamx.common.ISE; -import com.metamx.common.guava.ResourceClosingSequence; -import com.metamx.common.guava.Sequence; -import com.metamx.common.guava.Sequences; -import com.metamx.common.guava.Yielder; -import com.metamx.common.guava.YieldingAccumulator; -import io.druid.client.cache.Cache; -import io.druid.client.cache.CacheConfig; -import io.druid.granularity.AllGranularity; -import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.Result; -import io.druid.query.SegmentDescriptor; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.query.aggregation.LongSumAggregatorFactory; -import io.druid.query.topn.TopNQueryBuilder; -import io.druid.query.topn.TopNQueryConfig; -import io.druid.query.topn.TopNQueryQueryToolChest; -import io.druid.query.topn.TopNResultValue; -import org.easymock.EasyMock; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - -public class CachePopulatingQueryRunnerTest -{ - - private static final List AGGS = Arrays.asList( - new CountAggregatorFactory("rows"), - new LongSumAggregatorFactory("imps", "imps"), - new LongSumAggregatorFactory("impers", "imps") - ); - - @Test - public void testCachePopulatingQueryRunnerResourceClosing() throws Exception - { - Iterable> expectedRes = makeTopNResults( - new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, - new DateTime("2011-01-06"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, - new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, - new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, - new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 - ); - final TopNQueryBuilder builder = new TopNQueryBuilder() - .dataSource("ds") - .dimension("top_dim") - .metric("imps") - .threshold(3) - .intervals("2011-01-05/2011-01-10") - .aggregators(AGGS) - .granularity(AllGranularity.ALL); - - final AssertingClosable closable = new AssertingClosable(); - final Sequence resultSeq = new ResourceClosingSequence( - Sequences.simple(expectedRes), closable - ) - { - @Override - public Yielder toYielder(Object initValue, YieldingAccumulator accumulator) - { - Assert.assertFalse(closable.isClosed()); - return super.toYielder( - initValue, - accumulator - ); - } - }; - - Cache cache = EasyMock.createMock(Cache.class); - // cache populater ignores populating for local cache, so a dummy cache - EasyMock.expect(cache.isLocal()).andReturn(false); - CachePopulatingQueryRunner runner = new CachePopulatingQueryRunner( - "segment", - new SegmentDescriptor(new Interval("2011/2012"), "version", 0), - new DefaultObjectMapper(), - cache, - new TopNQueryQueryToolChest(new TopNQueryConfig()), - new QueryRunner() - { - @Override - public Sequence run(Query query) - { - return resultSeq; - } - }, - new CacheConfig() - - ); - - Sequence res = runner.run(builder.build()); - // base sequence is not closed yet - Assert.assertTrue(closable.isClosed()); - ArrayList results = Sequences.toList(res, new ArrayList()); - Assert.assertTrue(closable.isClosed()); - Assert.assertEquals(expectedRes, results); - - } - - private Iterable> makeTopNResults - (Object... objects) - { - List> retVal = Lists.newArrayList(); - int index = 0; - while (index < objects.length) { - DateTime timestamp = (DateTime) objects[index++]; - - List> values = Lists.newArrayList(); - while (index < objects.length && !(objects[index] instanceof DateTime)) { - if (objects.length - index < 3) { - throw new ISE( - "expect 3 values for each entry in the top list, had %d values left.", objects.length - index - ); - } - final double imps = ((Number) objects[index + 2]).doubleValue(); - final double rows = ((Number) objects[index + 1]).doubleValue(); - values.add( - ImmutableMap.of( - "top_dim", objects[index], - "rows", rows, - "imps", imps, - "impers", imps, - "avg_imps_per_row", imps / rows - ) - ); - index += 3; - } - - retVal.add(new Result<>(timestamp, new TopNResultValue(values))); - } - return retVal; - } - - private static class AssertingClosable implements Closeable - { - - private final AtomicBoolean closed = new AtomicBoolean(false); - - @Override - public void close() throws IOException - { - Assert.assertFalse(closed.get()); - Assert.assertTrue(closed.compareAndSet(false, true)); - } - - public boolean isClosed() - { - return closed.get(); - } - } - -} diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index cfea29f9a8e..582d0c648cc 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -47,6 +47,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.query.BySegmentResultValueClass; import io.druid.query.DataSource; import io.druid.query.Druids; +import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; import io.druid.query.QueryConfig; @@ -60,6 +61,7 @@ import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; import io.druid.query.aggregation.post.FieldAccessPostAggregator; import io.druid.query.filter.DimFilter; import io.druid.query.search.SearchQueryQueryToolChest; @@ -115,17 +117,22 @@ import java.util.concurrent.Executor; @RunWith(Parameterized.class) public class CachingClusteredClientTest { + public static final ImmutableMap CONTEXT = ImmutableMap.of("finalize", false); + + public static final MultipleIntervalSegmentSpec SEG_SPEC = new MultipleIntervalSegmentSpec(ImmutableList.of()); + public static final String DATA_SOURCE = "test"; + protected static final DefaultObjectMapper jsonMapper = new DefaultObjectMapper(new SmileFactory()); + + static { + jsonMapper.getFactory().setCodec(jsonMapper); + } + /** * We want a deterministic test, but we'd also like a bit of randomness for the distribution of segments * across servers. Thus, we loop multiple times and each time use a deterministically created Random instance. * Increase this value to increase exposure to random situations at the expense of test run time. */ private static final int RANDOMNESS = 10; - - public static final ImmutableMap CONTEXT = ImmutableMap.of(); - public static final MultipleIntervalSegmentSpec SEG_SPEC = new MultipleIntervalSegmentSpec(ImmutableList.of()); - public static final String DATA_SOURCE = "test"; - private static final List AGGS = Arrays.asList( new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("imps", "imps"), @@ -139,6 +146,22 @@ public class CachingClusteredClientTest new FieldAccessPostAggregator("imps", "imps"), new FieldAccessPostAggregator("rows", "rows") ) + ), + new ArithmeticPostAggregator( + "avg_imps_per_row_double", + "*", + Arrays.asList( + new FieldAccessPostAggregator("avg_imps_per_row", "avg_imps_per_row"), + new ConstantPostAggregator("constant", 2, 2 ) + ) + ), + new ArithmeticPostAggregator( + "avg_imps_per_row_half", + "/", + Arrays.asList( + new FieldAccessPostAggregator("avg_imps_per_row", "avg_imps_per_row"), + new ConstantPostAggregator("constant", 2, 2 ) + ) ) ); private static final List RENAMED_AGGS = Arrays.asList( @@ -152,6 +175,17 @@ public class CachingClusteredClientTest private static final DateTimeZone TIMEZONE = DateTimeZone.forID("America/Los_Angeles"); private static final QueryGranularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE); private static final String TOP_DIM = "a_dim"; + private final Random random; + protected VersionedIntervalTimeline timeline; + protected TimelineServerView serverView; + protected Cache cache; + public CachingClusteredClient client; + DruidServer[] servers; + + public CachingClusteredClientTest(int randomSeed) + { + this.random = new Random(randomSeed); + } @Parameterized.Parameters public static Collection constructorFeeder() throws IOException @@ -169,28 +203,6 @@ public class CachingClusteredClientTest ); } - - protected static final DefaultObjectMapper jsonMapper = new DefaultObjectMapper(new SmileFactory()); - - static { - jsonMapper.getFactory().setCodec(jsonMapper); - } - - private final Random random; - - protected VersionedIntervalTimeline timeline; - protected TimelineServerView serverView; - protected Cache cache; - - CachingClusteredClient client; - - DruidServer[] servers; - - public CachingClusteredClientTest(int randomSeed) - { - this.random = new Random(randomSeed); - } - @Before public void setUp() throws Exception { @@ -214,15 +226,18 @@ public class CachingClusteredClientTest public void testTimeseriesCaching() throws Exception { final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SOURCE) - .intervals(SEG_SPEC) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .aggregators(AGGS) - .postAggregators(POST_AGGS) - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + QueryRunner runner = new FinalizeResultsQueryRunner(client, new TimeseriesQueryQueryToolChest(new QueryConfig())); testQueryCaching( + runner, builder.build(), new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000), new Interval("2011-01-02/2011-01-03"), makeTimeResults(new DateTime("2011-01-02"), 30, 6000), @@ -263,11 +278,11 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), 18, 521, new DateTime("2011-01-09T01"), 181, 52 ), - client.run( + runner.run( builder.intervals("2011-01-01/2011-01-10") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS) - .build() + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() ) ); } @@ -277,15 +292,18 @@ public class CachingClusteredClientTest public void testTimeseriesCachingTimeZone() throws Exception { final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SOURCE) - .intervals(SEG_SPEC) - .filters(DIM_FILTER) - .granularity(PT1H_TZ_GRANULARITY) - .aggregators(AGGS) - .postAggregators(POST_AGGS) - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(PT1H_TZ_GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + QueryRunner runner = new FinalizeResultsQueryRunner(client, new TimeseriesQueryQueryToolChest(new QueryConfig())); testQueryCaching( + runner, builder.build(), new Interval("2011-11-04/2011-11-08"), makeTimeResults( @@ -303,11 +321,11 @@ public class CachingClusteredClientTest new DateTime("2011-11-06", TIMEZONE), 23, 85312, new DateTime("2011-11-07", TIMEZONE), 85, 102 ), - client.run( + runner.run( builder.intervals("2011-11-04/2011-11-08") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS) - .build() + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() ) ); } @@ -316,18 +334,24 @@ public class CachingClusteredClientTest public void testDisableUseCache() throws Exception { final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SOURCE) - .intervals(SEG_SPEC) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .aggregators(AGGS) - .postAggregators(POST_AGGS); - + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + QueryRunner runner = new FinalizeResultsQueryRunner(client, new TimeseriesQueryQueryToolChest(new QueryConfig())); testQueryCaching( + runner, 1, true, - builder.context(ImmutableMap.of("useCache", "false", - "populateCache", "true")).build(), + builder.context( + ImmutableMap.of( + "useCache", "false", + "populateCache", "true" + ) + ).build(), new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000) ); @@ -338,10 +362,15 @@ public class CachingClusteredClientTest cache.close("0_0"); testQueryCaching( + runner, 1, false, - builder.context(ImmutableMap.of("useCache", "false", - "populateCache", "false")).build(), + builder.context( + ImmutableMap.of( + "useCache", "false", + "populateCache", "false" + ) + ).build(), new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000) ); @@ -350,10 +379,15 @@ public class CachingClusteredClientTest Assert.assertEquals(0, cache.getStats().getNumMisses()); testQueryCaching( + client, 1, false, - builder.context(ImmutableMap.of("useCache", "true", - "populateCache", "false")).build(), + builder.context( + ImmutableMap.of( + "useCache", "true", + "populateCache", "false" + ) + ).build(), new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000) ); @@ -378,7 +412,10 @@ public class CachingClusteredClientTest .postAggregators(POST_AGGS) .context(CONTEXT); + QueryRunner runner = new FinalizeResultsQueryRunner(client, new TopNQueryQueryToolChest(new TopNQueryConfig())); + testQueryCaching( + runner, builder.build(), new Interval("2011-01-01/2011-01-02"), makeTopNResults(new DateTime("2011-01-01"), "a", 50, 5000, "b", 50, 4999, "c", 50, 4998), @@ -392,7 +429,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-06"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, - new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + new DateTime("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983 ), new Interval("2011-01-05/2011-01-10"), @@ -401,7 +438,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-06T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, - new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983 ) ); @@ -417,15 +454,15 @@ public class CachingClusteredClientTest new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, - new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983, - new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + new DateTime("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983, + new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983 ), - client.run( + runner.run( builder.intervals("2011-01-01/2011-01-10") - .metric("imps") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS) - .build() + .metric("imps") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() ) ); } @@ -446,7 +483,10 @@ public class CachingClusteredClientTest .postAggregators(POST_AGGS) .context(CONTEXT); + QueryRunner runner = new FinalizeResultsQueryRunner(client, new TopNQueryQueryToolChest(new TopNQueryConfig())); + testQueryCaching( + runner, builder.build(), new Interval("2011-11-04/2011-11-08"), makeTopNResults( @@ -465,12 +505,12 @@ public class CachingClusteredClientTest new DateTime("2011-11-06", TIMEZONE), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986 ), - client.run( + runner.run( builder.intervals("2011-11-04/2011-11-08") - .metric("imps") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS) - .build() + .metric("imps") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() ) ); } @@ -491,7 +531,9 @@ public class CachingClusteredClientTest .postAggregators(POST_AGGS) .context(CONTEXT); + QueryRunner runner = new FinalizeResultsQueryRunner(client, new TopNQueryQueryToolChest(new TopNQueryConfig())); testQueryCaching( + runner, builder.build(), new Interval("2011-01-01/2011-01-02"), makeTopNResults(), @@ -518,6 +560,7 @@ public class CachingClusteredClientTest ) ); + TestHelper.assertExpectedResults( makeRenamedTopNResults( new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, @@ -531,12 +574,79 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983, new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 ), - client.run( + runner.run( builder.intervals("2011-01-01/2011-01-10") - .metric("imps") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS) - .build() + .metric("imps") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() + ) + ); + } + + @Test + public void testTopNOnPostAggMetricCaching() { + final TopNQueryBuilder builder = new TopNQueryBuilder() + .dataSource(DATA_SOURCE) + .dimension(TOP_DIM) + .metric("avg_imps_per_row_double") + .threshold(3) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + QueryRunner runner = new FinalizeResultsQueryRunner(client, new TopNQueryQueryToolChest(new TopNQueryConfig())); + testQueryCaching( + runner, + builder.build(), + new Interval("2011-01-01/2011-01-02"), + makeTopNResults(), + + new Interval("2011-01-02/2011-01-03"), + makeTopNResults(), + + new Interval("2011-01-05/2011-01-10"), + makeTopNResults( + new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ), + + new Interval("2011-01-05/2011-01-10"), + makeTopNResults( + new DateTime("2011-01-05T01"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ) + ); + + + TestHelper.assertExpectedResults( + makeTopNResults( + new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-05T01"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-06T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983, + new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ), + runner.run( + builder.intervals("2011-01-01/2011-01-10") + .metric("avg_imps_per_row_double") + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .build() ) ); } @@ -545,6 +655,7 @@ public class CachingClusteredClientTest public void testSearchCaching() throws Exception { testQueryCaching( + client, new SearchQuery( new TableDataSource(DATA_SOURCE), DIM_FILTER, @@ -582,13 +693,14 @@ public class CachingClusteredClientTest ); } - public void testQueryCaching(final Query query, Object... args) + public void testQueryCaching(QueryRunner runner, final Query query, Object... args) { - testQueryCaching(3, true, query, args); + testQueryCaching(runner, 3, true, query, args); } @SuppressWarnings("unchecked") public void testQueryCaching( + final QueryRunner runner, final int numTimesToQuery, boolean expectBySegment, final Query query, Object... args // does this assume query intervals must be ordered? @@ -638,8 +750,8 @@ public class CachingClusteredClientTest EasyMock.expect(serverView.getQueryRunner(server)) - .andReturn(expectations.getQueryRunner()) - .once(); + .andReturn(expectations.getQueryRunner()) + .once(); final Capture capture = new Capture(); queryCaptures.add(capture); @@ -656,8 +768,8 @@ public class CachingClusteredClientTest } EasyMock.expect(queryable.run(EasyMock.capture(capture))) - .andReturn(toQueryableTimeseriesResults(expectBySegment, segmentIds, intervals, results)) - .once(); + .andReturn(toQueryableTimeseriesResults(expectBySegment, segmentIds, intervals, results)) + .once(); } else if (query instanceof TopNQuery) { List segmentIds = Lists.newArrayList(); @@ -669,8 +781,8 @@ public class CachingClusteredClientTest results.add(expectation.getResults()); } EasyMock.expect(queryable.run(EasyMock.capture(capture))) - .andReturn(toQueryableTopNResults(segmentIds, intervals, results)) - .once(); + .andReturn(toQueryableTopNResults(segmentIds, intervals, results)) + .once(); } else if (query instanceof SearchQuery) { List segmentIds = Lists.newArrayList(); List intervals = Lists.newArrayList(); @@ -681,8 +793,8 @@ public class CachingClusteredClientTest results.add(expectation.getResults()); } EasyMock.expect(queryable.run(EasyMock.capture(capture))) - .andReturn(toQueryableSearchResults(segmentIds, intervals, results)) - .once(); + .andReturn(toQueryableSearchResults(segmentIds, intervals, results)) + .once(); } else if (query instanceof TimeBoundaryQuery) { List segmentIds = Lists.newArrayList(); List intervals = Lists.newArrayList(); @@ -693,8 +805,8 @@ public class CachingClusteredClientTest results.add(expectation.getResults()); } EasyMock.expect(queryable.run(EasyMock.capture(capture))) - .andReturn(toQueryableTimeBoundaryResults(segmentIds, intervals, results)) - .once(); + .andReturn(toQueryableTimeBoundaryResults(segmentIds, intervals, results)) + .once(); } else { throw new ISE("Unknown query type[%s]", query.getClass()); } @@ -742,7 +854,7 @@ public class CachingClusteredClientTest } ) ), - client.run( + runner.run( query.withQuerySegmentSpec( new MultipleIntervalSegmentSpec( Arrays.asList( @@ -762,11 +874,11 @@ public class CachingClusteredClientTest for (Capture queryCapture : queryCaptures) { Query capturedQuery = (Query) queryCapture.getValue(); if (expectBySegment) { - Assert.assertEquals("true", capturedQuery.getContextValue("bySegment")); + Assert.assertEquals(true, capturedQuery.getContextValue("bySegment")); } else { Assert.assertTrue( capturedQuery.getContextValue("bySegment") == null || - capturedQuery.getContextValue("bySegment").equals("false") + capturedQuery.getContextValue("bySegment").equals(false) ); } } @@ -979,20 +1091,22 @@ public class CachingClusteredClientTest List> retVal = Lists.newArrayListWithCapacity(objects.length / 3); for (int i = 0; i < objects.length; i += 3) { + double avg_impr = ((Number) objects[i + 2]).doubleValue() / ((Number) objects[i + 1]).doubleValue(); retVal.add( new Result<>( (DateTime) objects[i], new TimeseriesResultValue( - ImmutableMap.of( - "rows", objects[i + 1], - "imps", objects[i + 2], - "impers", objects[i + 2], - "avg_imps_per_row", - ((Number) objects[i + 2]).doubleValue() / ((Number) objects[i + 1]).doubleValue() + ImmutableMap.builder() + .put("rows", objects[i + 1]) + .put("imps", objects[i + 2]) + .put("impers", objects[i + 2]) + .put("avg_imps_per_row",avg_impr) + .put("avg_imps_per_row_half",avg_impr / 2) + .put("avg_imps_per_row_double",avg_impr * 2) + .build() ) ) - ) - ); + ); } return retVal; } @@ -1071,13 +1185,15 @@ public class CachingClusteredClientTest final double imps = ((Number) objects[index + 2]).doubleValue(); final double rows = ((Number) objects[index + 1]).doubleValue(); values.add( - ImmutableMap.of( - TOP_DIM, objects[index], - "rows", rows, - "imps", imps, - "impers", imps, - "avg_imps_per_row", imps / rows - ) + ImmutableMap.builder() + .put(TOP_DIM, objects[index]) + .put("rows", rows) + .put("imps", imps) + .put("impers", imps) + .put("avg_imps_per_row", imps / rows) + .put("avg_imps_per_row_double", ((imps * 2) / rows)) + .put("avg_imps_per_row_half", (imps / (rows * 2))) + .build() ); index += 3; } @@ -1160,13 +1276,13 @@ public class CachingClusteredClientTest return new CachingClusteredClient( new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() - .put( - TimeseriesQuery.class, - new TimeseriesQueryQueryToolChest(new QueryConfig()) - ) - .put(TopNQuery.class, new TopNQueryQueryToolChest(new TopNQueryConfig())) - .put(SearchQuery.class, new SearchQueryQueryToolChest(new SearchQueryConfig())) - .build() + .put( + TimeseriesQuery.class, + new TimeseriesQueryQueryToolChest(new QueryConfig()) + ) + .put(TopNQuery.class, new TopNQueryQueryToolChest(new TopNQueryConfig())) + .put(SearchQuery.class, new SearchQueryQueryToolChest(new SearchQueryConfig())) + .build() ), new TimelineServerView() { @@ -1241,6 +1357,8 @@ public class CachingClusteredClientTest private class MyDataSegment extends DataSegment { + private final DataSegment baseSegment = segment; + private MyDataSegment() { super( @@ -1256,8 +1374,6 @@ public class CachingClusteredClientTest ); } - private final DataSegment baseSegment = segment; - @Override @JsonProperty public String getDataSource() @@ -1358,7 +1474,6 @@ public class CachingClusteredClientTest { private final DruidServer server; private final QueryRunner queryRunner; - private final List expectations = Lists.newArrayList(); public ServerExpectations( diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java new file mode 100644 index 00000000000..d72da61e751 --- /dev/null +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -0,0 +1,293 @@ +/* + * 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.client; + +import com.google.common.base.Function; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.metamx.common.ISE; +import com.metamx.common.guava.ResourceClosingSequence; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.common.guava.Yielder; +import com.metamx.common.guava.YieldingAccumulator; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; +import io.druid.client.cache.MapCache; +import io.druid.granularity.AllGranularity; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.CacheStrategy; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.Result; +import io.druid.query.SegmentDescriptor; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +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.TopNResultValue; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +public class CachingQueryRunnerTest +{ + + private static final List AGGS = Arrays.asList( + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("imps", "imps"), + new LongSumAggregatorFactory("impers", "imps") + ); + + private static final Object[] objects = new Object[]{ + new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + }; + + @Test + public void testCloseAndPopulate() throws Exception + { + Iterable> expectedRes = makeTopNResults(false,objects); + final TopNQueryBuilder builder = new TopNQueryBuilder() + .dataSource("ds") + .dimension("top_dim") + .metric("imps") + .threshold(3) + .intervals("2011-01-05/2011-01-10") + .aggregators(AGGS) + .granularity(AllGranularity.ALL); + + final AssertingClosable closable = new AssertingClosable(); + final Sequence resultSeq = new ResourceClosingSequence( + Sequences.simple(expectedRes), closable + ) + { + @Override + public Yielder toYielder(Object initValue, YieldingAccumulator accumulator) + { + Assert.assertFalse(closable.isClosed()); + return super.toYielder( + initValue, + accumulator + ); + } + }; + + Cache cache = MapCache.create(1024 * 1024); + + String segmentIdentifier = "segment"; + SegmentDescriptor segmentDescriptor = new SegmentDescriptor(new Interval("2011/2012"), "version", 0); + + TopNQueryQueryToolChest toolchest = new TopNQueryQueryToolChest(new TopNQueryConfig()); + DefaultObjectMapper objectMapper = new DefaultObjectMapper(); + CachingQueryRunner runner = new CachingQueryRunner( + segmentIdentifier, + segmentDescriptor, + objectMapper, + cache, + toolchest, + new QueryRunner() + { + @Override + public Sequence run(Query query) + { + return resultSeq; + } + }, + new CacheConfig() + + ); + + TopNQuery query = builder.build(); + CacheStrategy, Object, TopNQuery> cacheStrategy = toolchest.getCacheStrategy(query); + Cache.NamedKey cacheKey = CacheUtil.computeSegmentCacheKey( + segmentIdentifier, + segmentDescriptor, + cacheStrategy.computeCacheKey(query) + ); + + + Sequence res = runner.run(query); + // base sequence is not closed yet + Assert.assertFalse("sequence must not be closed", closable.isClosed()); + Assert.assertNull("cache must be empty", cache.get(cacheKey)); + + ArrayList results = Sequences.toList(res, new ArrayList()); + Assert.assertTrue(closable.isClosed()); + Assert.assertEquals(expectedRes, results); + + Iterable> expectedCacheRes = makeTopNResults(true, objects); + + byte[] cacheValue = cache.get(cacheKey); + Assert.assertNotNull(cacheValue); + + Function> fn = cacheStrategy.pullFromCache(); + List> cacheResults = Lists.newArrayList( + Iterators.transform( + objectMapper.readValues( + objectMapper.getFactory().createParser(cacheValue), + cacheStrategy.getCacheObjectClazz() + ), + fn + ) + ); + Assert.assertEquals(expectedCacheRes, cacheResults); + } + + @Test + public void testUseCache() throws Exception + { + DefaultObjectMapper objectMapper = new DefaultObjectMapper(); + Iterable> expectedResults = makeTopNResults(true, objects); + String segmentIdentifier = "segment"; + SegmentDescriptor segmentDescriptor = new SegmentDescriptor(new Interval("2011/2012"), "version", 0); + TopNQueryQueryToolChest toolchest = new TopNQueryQueryToolChest(new TopNQueryConfig()); + + final TopNQueryBuilder builder = new TopNQueryBuilder() + .dataSource("ds") + .dimension("top_dim") + .metric("imps") + .threshold(3) + .intervals("2011-01-05/2011-01-10") + .aggregators(AGGS) + .granularity(AllGranularity.ALL); + + final TopNQuery query = builder.build(); + + CacheStrategy, Object, TopNQuery> cacheStrategy = toolchest.getCacheStrategy(query); + Cache.NamedKey cacheKey = CacheUtil.computeSegmentCacheKey( + segmentIdentifier, + segmentDescriptor, + cacheStrategy.computeCacheKey(query) + ); + + Cache cache = MapCache.create(1024 * 1024); + CacheUtil.populate( + cache, + objectMapper, + cacheKey, + Iterables.transform(expectedResults, cacheStrategy.prepareForCache()) + ); + + CachingQueryRunner runner = new CachingQueryRunner( + segmentIdentifier, + segmentDescriptor, + objectMapper, + cache, + toolchest, + // return an empty sequence since results should get pulled from cache + new QueryRunner() + { + @Override + public Sequence run(Query query) + { + return Sequences.empty(); + } + }, + new CacheConfig() + + ); + + List results = Sequences.toList(runner.run(query), new ArrayList()); + Assert.assertEquals(expectedResults, results); + } + + private Iterable> makeTopNResults + (boolean cachedResults, Object... objects) + { + List> retVal = Lists.newArrayList(); + int index = 0; + while (index < objects.length) { + DateTime timestamp = (DateTime) objects[index++]; + + List> values = Lists.newArrayList(); + while (index < objects.length && !(objects[index] instanceof DateTime)) { + if (objects.length - index < 3) { + throw new ISE( + "expect 3 values for each entry in the top list, had %d values left.", objects.length - index + ); + } + final double imps = ((Number) objects[index + 2]).doubleValue(); + final double rows = ((Number) objects[index + 1]).doubleValue(); + + if (cachedResults) { + values.add( + ImmutableMap.of( + "top_dim", objects[index], + "rows", rows, + "imps", imps, + "impers", imps + ) + ); + } else { + values.add( + ImmutableMap.of( + "top_dim", objects[index], + "rows", rows, + "imps", imps, + "impers", imps, + "avg_imps_per_row", imps / rows + ) + ); + } + index += 3; + } + + retVal.add(new Result<>(timestamp, new TopNResultValue(values))); + } + return retVal; + } + + private static class AssertingClosable implements Closeable + { + + private final AtomicBoolean closed = new AtomicBoolean(false); + + @Override + public void close() throws IOException + { + Assert.assertFalse(closed.get()); + Assert.assertTrue(closed.compareAndSet(false, true)); + } + + public boolean isClosed() + { + return closed.get(); + } + } + +} diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 223c720c7d8..431fdce8318 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -578,7 +578,7 @@ public class ServerManagerTest } @Override - public Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn) + public Function makePreComputeManipulatorFn(QueryType query, MetricManipulationFn fn) { return Functions.identity(); } diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java new file mode 100644 index 00000000000..38276d69623 --- /dev/null +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -0,0 +1,209 @@ +/* + * 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; + +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.guice.Jerseys; +import io.druid.guice.LazySingleton; +import io.druid.guice.annotations.Global; +import io.druid.initialization.Initialization; +import io.druid.server.initialization.JettyServerInitializer; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.DefaultHandler; +import org.eclipse.jetty.server.handler.HandlerList; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.GzipFilter; +import org.junit.Ignore; +import org.junit.Test; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Response; +import java.net.URL; +import java.nio.charset.Charset; +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +public class JettyTest +{ + public static void setProperties() + { + System.setProperty("druid.host", "localhost:9999"); + System.setProperty("druid.port", "9999"); + System.setProperty("druid.server.http.numThreads", "20"); + System.setProperty("druid.service", "test"); + System.setProperty("druid.server.http.maxIdleTime", "PT1S"); + System.setProperty("druid.global.http.readTimeout", "PT1S"); + } + + @Test @Ignore // this test will deadlock if it hits an issue, so ignored by default + public void testTimeouts() throws Exception + { + // test for request timeouts properly not locking up all threads + + setProperties(); + Injector injector = Initialization.makeInjectorWithModules( + Initialization.makeStartupInjector(), Lists.newArrayList( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(JettyServerInitializer.class).to(JettyServerInit.class).in(LazySingleton.class); + Jerseys.addResource(binder, SlowResource.class); + + } + } + ) + ); + Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + // Jetty is Lazy Initialized do a getInstance + injector.getInstance(Server.class); + lifecycle.start(); + ClientHolder holder = injector.getInstance(ClientHolder.class); + final HttpClient client = holder.getClient(); + final Executor executor = Executors.newFixedThreadPool(100); + final AtomicLong count = new AtomicLong(0); + final CountDownLatch latch = new CountDownLatch(1000); + for (int i = 0; i < 10000; i++) { + executor.execute( + new Runnable() + { + @Override + public void run() + { + executor.execute( + new Runnable() + { + @Override + public void run() + { + long startTime = System.currentTimeMillis(); + long startTime2 = 0; + try { + ListenableFuture go = client.get( + new URL( + "http://localhost:9999/slow/hello" + ) + ) + .go(new StatusResponseHandler(Charset.defaultCharset())); + startTime2 = System.currentTimeMillis(); + go.get(); + } + catch (Exception e) { + e.printStackTrace(); + } + finally { + System.out + .println( + "Response time client" + + (System.currentTimeMillis() - startTime) + + "time taken for getting future" + + (System.currentTimeMillis() - startTime2) + + "Counter " + count.incrementAndGet() + ); + latch.countDown(); + + } + } + } + ); + } + } + ); + } + + latch.await(); + lifecycle.stop(); + + } + + public static class ClientHolder + { + HttpClient client; + + @Inject + ClientHolder(@Global HttpClient client) + { + this.client = client; + } + + public HttpClient getClient() + { + return client; + } + } + + public static class JettyServerInit implements JettyServerInitializer + { + + @Override + public void initialize(Server server, Injector injector) + { + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(GuiceFilter.class, "/*", null); + + final HandlerList handlerList = new HandlerList(); + handlerList.setHandlers(new Handler[]{root, new DefaultHandler()}); + server.setHandler(handlerList); + } + } + + @Path("/slow") + public static class SlowResource + { + + public static Random random = new Random(); + + @GET + @Path("/hello") + @Produces("application/json") + public Response hello() + { + try { + TimeUnit.MILLISECONDS.sleep(100 + random.nextInt(2000)); + } + catch (InterruptedException e) { + // + } + return Response.ok("hello").build(); + } + } +} diff --git a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java index b2a22af38ec..6d058d200fa 100644 --- a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java +++ b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java @@ -142,19 +142,18 @@ public class TieredBrokerHostSelectorTest @Test public void testSelectMatchesNothing() throws Exception { - Pair retVal = brokerSelector.select( + String brokerName = (String) brokerSelector.select( Druids.newTimeseriesQueryBuilder() .dataSource("test") .granularity("all") .aggregators(Arrays.asList(new CountAggregatorFactory("rows"))) .intervals(Arrays.asList(new Interval("2010-08-31/2010-09-01"))) .build() - ); + ).lhs; - Assert.assertEquals(null, retVal); + Assert.assertEquals("hotBroker", brokerName); } - @Test public void testSelectMultiInterval() throws Exception { diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index 9b30768ca40..bfd573dae89 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -19,26 +19,23 @@ package io.druid.server.shard; -import com.fasterxml.jackson.databind.BeanProperty; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.InjectableValues; import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.metamx.common.ISE; import io.druid.TestUtil; +import io.druid.data.input.InputRow; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.PartitionChunk; import io.druid.timeline.partition.ShardSpec; import junit.framework.Assert; -import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; import java.util.List; public class HashBasedNumberedShardSpecTest { - @Test public void testSerdeRoundTrip() throws Exception { @@ -107,4 +104,94 @@ public class HashBasedNumberedShardSpecTest Assert.assertFalse(chunks.get(2).abuts(chunks.get(2))); } + @Test + public void testIsInChunk() + { + + List specs = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + specs.add(new HashOverridenShardSpec(i, 3)); + } + + + assertExistsInOneSpec(specs, new HashInputRow(Integer.MIN_VALUE)); + assertExistsInOneSpec(specs, new HashInputRow(Integer.MAX_VALUE)); + assertExistsInOneSpec(specs, new HashInputRow(0)); + assertExistsInOneSpec(specs, new HashInputRow(1000)); + assertExistsInOneSpec(specs, new HashInputRow(-1000)); + + } + + public boolean assertExistsInOneSpec(List specs, InputRow row) + { + for (ShardSpec spec : specs) { + if (spec.isInChunk(row)) { + return true; + } + } + throw new ISE("None of the partition matches"); + } + + public static class HashOverridenShardSpec extends HashBasedNumberedShardSpec + { + public HashOverridenShardSpec( + int partitionNum, + int partitions + ) + { + super(partitionNum, partitions); + } + + @Override + protected int hash(InputRow inputRow) + { + return inputRow.hashCode(); + } + } + + public static class HashInputRow implements InputRow + { + private final int hashcode; + + HashInputRow(int hashcode) + { + this.hashcode = hashcode; + } + + @Override + public int hashCode() + { + return hashcode; + } + + @Override + public List getDimensions() + { + return null; + } + + @Override + public long getTimestampFromEpoch() + { + return 0; + } + + @Override + public List getDimension(String s) + { + return null; + } + + @Override + public Object getRaw(String s) + { + return null; + } + + @Override + public float getFloatMetric(String s) + { + return 0; + } + } } diff --git a/services/pom.xml b/services/pom.xml index 2029aed81fe..90ef6c887c8 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.83-SNAPSHOT + 0.6.102-SNAPSHOT diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 88582af0947..308ed761414 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -55,7 +55,7 @@ import java.util.List; */ @Command( name = "broker", - description = "Runs a broker node, see http://druid.io/docs/0.6.81/Broker.html for a description" + description = "Runs a broker node, see http://druid.io/docs/latest/Broker.html for a description" ) public class CliBroker extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 2b2b17ec0fd..1c551c3e402 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -66,7 +66,7 @@ import java.util.List; */ @Command( name = "coordinator", - description = "Runs the Coordinator, see http://druid.io/docs/0.6.81/Coordinator.html for a description." + description = "Runs the Coordinator, see http://druid.io/docs/latest/Coordinator.html for a description." ) public class CliCoordinator extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index 838d001a714..cf81cc7aca5 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -41,22 +41,25 @@ import java.util.List; */ @Command( name = "hadoop", - description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.81/Batch-ingestion.html for a description." + description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/latest/Batch-ingestion.html for a description." ) public class CliHadoopIndexer implements Runnable { + + private static String defaultHadoopCoordinates = "org.apache.hadoop:hadoop-client:2.3.0"; + private static final Logger log = new Logger(CliHadoopIndexer.class); @Arguments(description = "A JSON object or the path to a file that contains a JSON object", required = true) private String argumentSpec; @Option(name = "hadoop", - description = "The maven coordinates to the version of hadoop to run with. Defaults to org.apache.hadoop:hadoop-core:1.0.3") - private String hadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3"; + description = "The maven coordinates to the version of hadoop to run with. Defaults to org.apache.hadoop:hadoop-client:2.3.0") + private String hadoopCoordinates = defaultHadoopCoordinates; @Option(name = "hadoopDependencies", - description = "The maven coordinates to the version of hadoop and all dependencies to run with. Defaults to using org.apache.hadoop:hadoop-core:1.0.3") - private List hadoopDependencyCoordinates = Arrays.asList("org.apache.hadoop:hadoop-core:1.0.3"); + description = "The maven coordinates to the version of hadoop and all dependencies to run with. Defaults to using org.apache.hadoop:hadoop-client:2.3.0") + private List hadoopDependencyCoordinates = Arrays.asList(defaultHadoopCoordinates); @Inject private ExtensionsConfig extensionsConfig = null; diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 3a41e35e22c..6d4152b9226 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -26,6 +26,7 @@ import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; +import io.druid.client.cache.CacheMonitor; import io.druid.client.cache.CacheProvider; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; @@ -38,6 +39,7 @@ import io.druid.server.QueryResource; import io.druid.server.coordination.ServerManager; import io.druid.server.coordination.ZkCoordinator; import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.metrics.MetricsModule; import org.eclipse.jetty.server.Server; import java.util.List; @@ -46,7 +48,7 @@ import java.util.List; */ @Command( name = "historical", - description = "Runs a Historical node, see http://druid.io/docs/0.6.81/Historical.html for a description" + description = "Runs a Historical node, see http://druid.io/docs/latest/Historical.html for a description" ) public class CliHistorical extends ServerRunnable { @@ -77,10 +79,11 @@ public class CliHistorical extends ServerRunnable LifecycleModule.register(binder, ZkCoordinator.class); LifecycleModule.register(binder, Server.class); + binder.bind(Cache.class).toProvider(CacheProvider.class).in(ManageLifecycle.class); JsonConfigProvider.bind(binder, "druid.historical.cache", CacheProvider.class); JsonConfigProvider.bind(binder, "druid.historical.cache", CacheConfig.class); - + MetricsModule.register(binder, CacheMonitor.class); } } ); diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index cbdbcb89b1f..9f95feb43ad 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -93,7 +93,7 @@ import java.util.List; */ @Command( name = "overlord", - description = "Runs an Overlord node, see http://druid.io/docs/0.6.81/Indexing-Service.html for a description" + description = "Runs an Overlord node, see http://druid.io/docs/latest/Indexing-Service.html for a description" ) public class CliOverlord extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index fcf55932d49..e80d22242c5 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -30,7 +30,7 @@ import java.util.List; */ @Command( name = "realtime", - description = "Runs a realtime node, see http://druid.io/docs/0.6.81/Realtime.html for a description" + description = "Runs a realtime node, see http://druid.io/docs/latest/Realtime.html for a description" ) public class CliRealtime extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index 1d0d10073bf..681ae78fa27 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -29,14 +29,17 @@ import io.druid.client.RoutingDruidClient; import io.druid.curator.discovery.DiscoveryModule; import io.druid.curator.discovery.ServerDiscoveryFactory; import io.druid.curator.discovery.ServerDiscoverySelector; +import io.druid.guice.HttpClientModule; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; +import io.druid.guice.annotations.Client; import io.druid.guice.annotations.Self; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.router.CoordinatorRuleManager; import io.druid.server.router.QueryHostFinder; +import io.druid.server.router.Router; import io.druid.server.router.TieredBrokerConfig; import io.druid.server.router.TieredBrokerHostSelector; import org.eclipse.jetty.server.Server; @@ -62,6 +65,7 @@ public class CliRouter extends ServerRunnable protected List getModules() { return ImmutableList.of( + new HttpClientModule("druid.router.http", Router.class), new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/PullDependencies.java b/services/src/main/java/io/druid/cli/PullDependencies.java index 1131a42a0b0..c6d7a84885d 100644 --- a/services/src/main/java/io/druid/cli/PullDependencies.java +++ b/services/src/main/java/io/druid/cli/PullDependencies.java @@ -19,7 +19,17 @@ package io.druid.cli; +import com.google.api.client.util.Lists; +import com.google.common.base.Throwables; +import com.google.inject.Inject; import io.airlift.command.Command; +import io.airlift.command.Option; +import io.druid.indexing.common.task.HadoopIndexTask; +import io.druid.initialization.Initialization; +import io.druid.server.initialization.ExtensionsConfig; +import io.tesla.aether.internal.DefaultTeslaAether; + +import java.util.List; @Command( @@ -28,8 +38,40 @@ import io.airlift.command.Command; ) public class PullDependencies implements Runnable { + @Option(name = "-c", + title = "coordinate", + description = "extra dependencies to pull down (e.g. hadoop coordinates)", + required = false) + public List coordinates; + + @Option(name = "--no-default-hadoop", + description = "don't pull down the default HadoopIndexTask dependencies", + required = false) + public boolean noDefaultHadoop; + + @Inject + public ExtensionsConfig extensionsConfig = null; + @Override - public void run() { - // dependencies are pulled down as a side-effect of Guice injection + public void run() + { + // Druid dependencies are pulled down as a side-effect of Guice injection. Extra dependencies are pulled down as + // a side-effect of getting class loaders. + final List allCoordinates = Lists.newArrayList(); + if (coordinates != null) { + allCoordinates.addAll(coordinates); + } + if (!noDefaultHadoop) { + allCoordinates.add(HadoopIndexTask.DEFAULT_HADOOP_COORDINATES); + } + try { + final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig); + for (final String coordinate : allCoordinates) { + Initialization.getClassLoaderForCoordinates(aetherClient, coordinate); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } } } diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index 8ed1849af5c..54c3a7f69e4 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -28,7 +28,6 @@ import io.druid.client.RoutingDruidClient; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.server.AsyncQueryForwardingServlet; -import io.druid.server.QueryIDProvider; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.log.RequestLogger; import io.druid.server.router.QueryHostFinder; @@ -51,7 +50,6 @@ public class RouterJettyServerInitializer implements JettyServerInitializer private final RoutingDruidClient routingDruidClient; private final ServiceEmitter emitter; private final RequestLogger requestLogger; - private final QueryIDProvider idProvider; @Inject public RouterJettyServerInitializer( @@ -60,8 +58,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer QueryHostFinder hostFinder, RoutingDruidClient routingDruidClient, ServiceEmitter emitter, - RequestLogger requestLogger, - QueryIDProvider idProvider + RequestLogger requestLogger ) { this.jsonMapper = jsonMapper; @@ -70,7 +67,6 @@ public class RouterJettyServerInitializer implements JettyServerInitializer this.routingDruidClient = routingDruidClient; this.emitter = emitter; this.requestLogger = requestLogger; - this.idProvider = idProvider; } @Override @@ -85,12 +81,12 @@ public class RouterJettyServerInitializer implements JettyServerInitializer hostFinder, routingDruidClient, emitter, - requestLogger, - idProvider + requestLogger ) ), "/druid/v2/*" ); queries.addFilter(GzipFilter.class, "/druid/v2/*", null); + queries.addFilter(GuiceFilter.class, "/status/*", null); final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new DefaultServlet()), "/*");