mirror of https://github.com/apache/druid.git
unit tests passingn at this point, finished rt port maybe
This commit is contained in:
commit
4b7c76762d
2
build.sh
2
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.52"
|
||||
echo "See also http://druid.io/docs/0.6.61"
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -82,6 +82,7 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim
|
|||
"segmentOutputPath": "s3n:\/\/billy-bucket\/the\/segments\/go\/here",
|
||||
"leaveIntermediate": "false",
|
||||
"partitionsSpec": {
|
||||
"type": "random"
|
||||
"targetPartitionSize": 5000000
|
||||
},
|
||||
"updaterJobSpec": {
|
||||
|
@ -145,12 +146,20 @@ The indexing process has the ability to roll data up as it processes the incomin
|
|||
|
||||
### Partitioning specification
|
||||
|
||||
Segments are always partitioned based on timestamp (according to the granularitySpec) and may be further partitioned in some other way. For example, data for a day may be split by the dimension "last\_name" into two segments: one with all values from A-M and one with all values from N-Z.
|
||||
Segments are always partitioned based on timestamp (according to the granularitySpec) and may be further partitioned in some other way depending on partition type.
|
||||
Druid supports two types of partitions spec - singleDimension and random.
|
||||
|
||||
In SingleDimension partition type data is partitioned based on the values in that dimension.
|
||||
For example, data for a day may be split by the dimension "last\_name" into two segments: one with all values from A-M and one with all values from N-Z.
|
||||
|
||||
In random 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.
|
||||
Random partition type is more efficient and gives better distribution of data.
|
||||
|
||||
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.
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|type of partitionSpec to be used |no, default : singleDimension|
|
||||
|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|
|
||||
|
|
|
@ -0,0 +1,116 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Broker Node Configuration
|
||||
=========================
|
||||
For general Broker Node information, see [here](Broker.html).
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server broker
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=broker
|
||||
druid.port=8080
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
```
|
||||
|
||||
Production Configs
|
||||
------------------
|
||||
These production configs are using S3 as a deep store.
|
||||
|
||||
JVM settings:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx#{HEAP_MAX}g
|
||||
-Xms#{HEAP_MIN}g
|
||||
-XX:NewSize=#{NEW_SIZE}g
|
||||
-XX:MaxNewSize=#{MAX_NEW_SIZE}g
|
||||
-XX:+UseConcMarkSweepGC
|
||||
-XX:+PrintGCDetails
|
||||
-XX:+PrintGCTimeStamps
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
-Djava.io.tmpdir=/mnt/tmp
|
||||
|
||||
-Dcom.sun.management.jmxremote.port=17071
|
||||
-Dcom.sun.management.jmxremote.authenticate=false
|
||||
-Dcom.sun.management.jmxremote.ssl=false
|
||||
```
|
||||
|
||||
Runtime.properties:
|
||||
|
||||
```
|
||||
druid.host=#{IP_ADDR}:8080
|
||||
druid.port=8080
|
||||
druid.service=druid/prod/broker
|
||||
|
||||
druid.zk.service.host=#{ZK_IPs}
|
||||
druid.zk.paths.base=/druid/prod
|
||||
|
||||
druid.discovery.curator.path=/prod/discovery
|
||||
|
||||
druid.broker.cache.type=memcached
|
||||
druid.broker.cache.hosts=#{MC_HOST1}:11211,#{MC_HOST2}:11211,#{MC_HOST3}:11211
|
||||
druid.broker.cache.expiration=2147483647
|
||||
druid.broker.cache.memcachedPrefix=d1
|
||||
druid.broker.http.numConnections=20
|
||||
druid.broker.http.readTimeout=PT5M
|
||||
|
||||
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"]
|
||||
|
||||
# Emit metrics over http
|
||||
druid.emitter=http
|
||||
druid.emitter.http.recipientBaseUrl=#{EMITTER_URL}
|
||||
|
||||
# If you choose to compress ZK announcements, you must do so for every node type
|
||||
druid.announcer.type=batch
|
||||
druid.curator.compress=true
|
||||
```
|
||||
|
||||
Runtime Configuration
|
||||
---------------------
|
||||
|
||||
The broker module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well:
|
||||
|
||||
|Property|Possible Values|Description|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|`druid.broker.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`|
|
||||
|`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to compute nodes. `random` choose randomly, `connectionCount` picks the node with the fewest number of active connections to|`random`|
|
||||
|
||||
#### Local Cache
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.broker.cache.sizeInBytes`|Maximum size of the cache. If this is zero, cache is disabled.|0|
|
||||
|`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|
|
||||
|
||||
#### 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|
|
|
@ -3,61 +3,11 @@ layout: doc_page
|
|||
---
|
||||
Broker
|
||||
======
|
||||
For Broker Node Configuration, see [Broker Configuration](Broker-Config.html).
|
||||
|
||||
The Broker is the node to route queries to if you want to run a distributed cluster. It understands the metadata published to ZooKeeper about what segments exist on what nodes and routes queries such that they hit the right nodes. This node also merges the result sets from all of the individual nodes together.
|
||||
On start up, Realtime nodes announce themselves and the segments they are serving in Zookeeper.
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server broker
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=broker
|
||||
druid.port=8080
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
```
|
||||
|
||||
JVM Configuration
|
||||
-----------------
|
||||
|
||||
The broker module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well:
|
||||
|
||||
|Property|Possible Values|Description|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|`druid.broker.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`|
|
||||
|`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to compute nodes. `random` choose randomly, `connectionCount` picks the node with the fewest number of active connections to|`random`|
|
||||
|
||||
#### Local Cache
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.broker.cache.sizeInBytes`|Maximum size of the cache. If this is zero, cache is disabled.|0|
|
||||
|`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|
|
||||
|
||||
#### 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|
|
||||
|
||||
Running
|
||||
-------
|
||||
|
||||
|
@ -65,7 +15,6 @@ Running
|
|||
io.druid.cli.Main server broker
|
||||
```
|
||||
|
||||
|
||||
Forwarding Queries
|
||||
------------------
|
||||
|
||||
|
|
|
@ -0,0 +1,150 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Coordinator Node Configuration
|
||||
==============================
|
||||
For general Coordinator Node information, see [here](Coordinator.html).
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server coordinator
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=coordinator
|
||||
druid.port=8082
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.coordinator.startDelay=PT60s
|
||||
```
|
||||
|
||||
Production Configs
|
||||
------------------
|
||||
These production configs are using S3 as a deep store.
|
||||
|
||||
JVM settings:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx#{HEAP_MAX}g
|
||||
-Xms#{HEAP_MIN}g
|
||||
-XX:NewSize=#{NEW_SIZE}g
|
||||
-XX:MaxNewSize=#{MAX_NEW_SIZE}g
|
||||
-XX:+UseConcMarkSweepGC
|
||||
-XX:+PrintGCDetails
|
||||
-XX:+PrintGCTimeStamps
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
-Djava.io.tmpdir=/mnt/tmp
|
||||
|
||||
-Dcom.sun.management.jmxremote.port=17071
|
||||
-Dcom.sun.management.jmxremote.authenticate=false
|
||||
-Dcom.sun.management.jmxremote.ssl=false
|
||||
```
|
||||
|
||||
Runtime.properties:
|
||||
|
||||
```
|
||||
druid.host=#{IP_ADDR}:8080
|
||||
druid.port=8080
|
||||
druid.service=druid/prod/coordinator
|
||||
|
||||
druid.zk.service.host=#{ZK_IPs}
|
||||
druid.zk.paths.base=/druid/prod
|
||||
|
||||
druid.discovery.curator.path=/prod/discovery
|
||||
|
||||
druid.db.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid
|
||||
druid.db.connector.user=#{MYSQL_USER}
|
||||
druid.db.connector.password=#{MYSQL_PW}
|
||||
druid.db.connector.useValidationQuery=true
|
||||
druid.db.tables.base=prod
|
||||
|
||||
druid.coordinator.period=PT60S
|
||||
druid.coordinator.period.indexingPeriod=PT1H
|
||||
druid.coordinator.startDelay=PT300S
|
||||
druid.coordinator.merge.on=false
|
||||
druid.coordinator.conversion.on=false
|
||||
|
||||
druid.selectors.indexing.serviceName=druid:prod:indexer
|
||||
|
||||
druid.monitoring.monitors=["com.metamx.metrics.SysMonitor", "com.metamx.metrics.JvmMonitor"]
|
||||
|
||||
# Emit metrics over http
|
||||
druid.emitter=http
|
||||
druid.emitter.http.recipientBaseUrl=#{EMITTER_URL}
|
||||
|
||||
# If you choose to compress ZK announcements, you must do so for every node type
|
||||
druid.announcer.type=batch
|
||||
druid.curator.compress=true
|
||||
```
|
||||
|
||||
Runtime Configuration
|
||||
---------------------
|
||||
|
||||
The coordinator module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.coordinator.period`|The run period for the coordinator. The coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S|
|
||||
|`druid.coordinator.period.indexingPeriod`|How often to send indexing tasks to the indexing service. Only applies if merge or conversion is turned on.|PT1800S (30 mins)|
|
||||
|`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S|
|
||||
|`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|PT300S|
|
||||
|`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false|
|
||||
|`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|15 minutes|
|
||||
|`druid.manager.segment.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments.|PT1M|
|
||||
|`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the coordinator to notice rules.|PT1M|
|
||||
|`druid.manager.rules.defaultTier`|The default tier from which default rules will be loaded from.|_default|
|
||||
|
||||
Dynamic Configuration
|
||||
---------------------
|
||||
|
||||
The coordinator has dynamic configuration to change certain behaviour on the fly. The coordinator a JSON spec object from the Druid [MySQL](MySQL.html) config table. This object is detailed below:
|
||||
|
||||
It is recommended that you use the Coordinator Console to configure these parameters. However, if you need to do it via HTTP, the JSON object can be submitted to the overlord via a POST request at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<PORT>/coordinator/config
|
||||
```
|
||||
|
||||
A sample worker setup spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"millisToWaitBeforeDeleting": 900000,
|
||||
"mergeBytesLimit": 100000000L,
|
||||
"mergeSegmentsLimit" : 1000,
|
||||
"maxSegmentsToMove": 5,
|
||||
"replicantLifetime": 15,
|
||||
"replicationThrottleLimit": 10,
|
||||
"emitBalancingStats": false
|
||||
}
|
||||
```
|
||||
|
||||
Issuing a GET request at the same URL will return the spec that is currently in place. A description of the config setup spec is shown below.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`millisToWaitBeforeDeleting`|How long does the coordinator need to be active before it can start deleting segments.|90000 (15 mins)|
|
||||
|`mergeBytesLimit`|The maximum number of bytes to merge (for segments).|100000000L|
|
||||
|`mergeSegmentsLimit`|The maximum number of segments that can be in a single merge [task](Tasks.html).|Integer.MAX_VALUE|
|
||||
|`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|5|
|
||||
|`replicantLifetime`|The maximum number of coordinator runs for a segment to be replicated before we start alerting.|15|
|
||||
|`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10|
|
||||
|`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false|
|
|
@ -1,8 +1,9 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Coordinator
|
||||
===========
|
||||
Coordinator Node
|
||||
================
|
||||
For Coordinator Node Configuration, see [Coordinator Configuration](Coordinator-Config.html).
|
||||
|
||||
The Druid coordinator node is primarily responsible for segment management and distribution. More specifically, the Druid coordinator node communicates to historical nodes to load or drop segments based on configurations. The Druid coordinator is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load.
|
||||
|
||||
|
@ -10,89 +11,6 @@ The Druid coordinator runs periodically and the time between each run is a confi
|
|||
|
||||
Before any unassigned segments are serviced by historical nodes, the available historical nodes for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always assigned to the nodes with least capacity to maintain a level of balance between nodes. The coordinator does not directly communicate with a historical node when assigning it a new segment; instead the coordinator creates some temporary information about the new segment under load queue path of the historical node. Once this request is seen, the historical node will load the segment and begin servicing it.
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server coordinator
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=coordinator
|
||||
druid.port=8082
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.coordinator.startDelay=PT60s
|
||||
```
|
||||
|
||||
JVM Configuration
|
||||
-----------------
|
||||
|
||||
The coordinator module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.coordinator.period`|The run period for the coordinator. The coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S|
|
||||
|`druid.coordinator.period.indexingPeriod`|How often to send indexing tasks to the indexing service. Only applies if merge or conversion is turned on.|PT1800S (30 mins)|
|
||||
|`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S|
|
||||
|`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|PT300S|
|
||||
|`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false|
|
||||
|`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|15 minutes|
|
||||
|`druid.manager.segment.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments.|PT1M|
|
||||
|`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the coordinator to notice rules.|PT1M|
|
||||
|`druid.manager.rules.defaultTier`|The default tier from which default rules will be loaded from.|_default|
|
||||
|
||||
Dynamic Configuration
|
||||
---------------------
|
||||
|
||||
The coordinator has dynamic configuration to change certain behaviour on the fly. The coordinator a JSON spec object from the Druid [MySQL](MySQL.html) config table. This object is detailed below:
|
||||
|
||||
It is recommended that you use the Coordinator Console to configure these parameters. However, if you need to do it via HTTP, the JSON object can be submitted to the overlord via a POST request at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<PORT>/coordinator/config
|
||||
```
|
||||
|
||||
A sample worker setup spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"millisToWaitBeforeDeleting": 900000,
|
||||
"mergeBytesLimit": 100000000L,
|
||||
"mergeSegmentsLimit" : 1000,
|
||||
"maxSegmentsToMove": 5,
|
||||
"replicantLifetime": 15,
|
||||
"replicationThrottleLimit": 10,
|
||||
"emitBalancingStats": false
|
||||
}
|
||||
```
|
||||
|
||||
Issuing a GET request at the same URL will return the spec that is currently in place. A description of the config setup spec is shown below.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`millisToWaitBeforeDeleting`|How long does the coordinator need to be active before it can start deleting segments.|90000 (15 mins)|
|
||||
|`mergeBytesLimit`|The maximum number of bytes to merge (for segments).|100000000L|
|
||||
|`mergeSegmentsLimit`|The maximum number of segments that can be in a single merge [task](Tasks.html).|Integer.MAX_VALUE|
|
||||
|`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|5|
|
||||
|`replicantLifetime`|The maximum number of coordinator runs for a segment to be replicated before we start alerting.|15|
|
||||
|`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10|
|
||||
|`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false|
|
||||
|
||||
### Running
|
||||
|
||||
```
|
||||
|
|
|
@ -63,7 +63,7 @@ If you use the [indexing service](Indexing-Service.html) for ingesting the data,
|
|||
|
||||
Specified here are the location of the datafile, the timestamp column, the format of the data, and the columns that will become dimensions in Druid.
|
||||
|
||||
Since the CSV data does not contain the column names, they will have to be added before that data can be processed:
|
||||
Since the CSV data cannot contain the column names (no header is allowed), these must be added before that data can be processed:
|
||||
|
||||
```json
|
||||
"firehose" : {
|
||||
|
|
|
@ -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.52
|
||||
git checkout druid-0.6.61
|
||||
./build.sh
|
||||
```
|
||||
|
||||
### Downloading the DSK (Druid Standalone Kit)
|
||||
|
||||
[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.52-bin.tar.gz) a stand-alone tarball and run it:
|
||||
[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.61-bin.tar.gz) a stand-alone tarball and run it:
|
||||
|
||||
``` bash
|
||||
tar -xzf druid-services-0.X.X-bin.tar.gz
|
||||
|
|
|
@ -0,0 +1,103 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Historical Node Configuration
|
||||
=============================
|
||||
For general Historical Node information, see [here](Historical.html).
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=historical
|
||||
druid.port=8081
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.server.maxSize=10000000000
|
||||
|
||||
# Change these to make Druid faster
|
||||
druid.processing.buffer.sizeBytes=100000000
|
||||
druid.processing.numThreads=1
|
||||
|
||||
druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 10000000000}]
|
||||
```
|
||||
|
||||
Note: This will spin up a Historical node with the local filesystem as deep storage.
|
||||
|
||||
Production Configs
|
||||
------------------
|
||||
These production configs are using S3 as a deep store.
|
||||
|
||||
JVM settings:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx#{HEAP_MAX}g
|
||||
-Xms#{HEAP_MIN}g
|
||||
-XX:NewSize=#{NEW_SIZE}g
|
||||
-XX:MaxNewSize=#{MAX_NEW_SIZE}g
|
||||
-XX:+UseConcMarkSweepGC
|
||||
-XX:+PrintGCDetails
|
||||
-XX:+PrintGCTimeStamps
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
-Djava.io.tmpdir=/mnt/tmp
|
||||
|
||||
-Dcom.sun.management.jmxremote.port=17071
|
||||
-Dcom.sun.management.jmxremote.authenticate=false
|
||||
-Dcom.sun.management.jmxremote.ssl=false
|
||||
```
|
||||
|
||||
Runtime.properties:
|
||||
|
||||
```
|
||||
druid.host=#{IP_ADDR}:8080
|
||||
druid.port=8080
|
||||
druid.service=druid/prod/compute/_default
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.61"]
|
||||
|
||||
druid.zk.service.host=#{ZK_IPs}
|
||||
druid.zk.paths.base=/druid/prod
|
||||
|
||||
druid.s3.accessKey=#{ACCESS_KEY}
|
||||
druid.s3.secretKey=#{SECRET_KEY}
|
||||
|
||||
druid.server.type=historical
|
||||
druid.server.maxSize=#{SERVER_MAXSIZE}
|
||||
druid.server.http.numThreads=50
|
||||
|
||||
druid.processing.buffer.sizeBytes=#{BUFFER_SIZE}}
|
||||
druid.processing.numThreads=#{NUM_THREADS}}
|
||||
|
||||
druid.segmentCache.locations=[{"path": "/mnt/persistent/zk_druid", "maxSize": #{SERVER_MAXSIZE}}]
|
||||
|
||||
druid.request.logging.type=file
|
||||
druid.request.logging.dir=request_logs/
|
||||
|
||||
druid.monitoring.monitors=["io.druid.server.metrics.ServerMonitor", "com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]
|
||||
|
||||
# Emit metrics over http
|
||||
druid.emitter=http
|
||||
druid.emitter.http.recipientBaseUrl=#{EMITTER_URL}
|
||||
|
||||
# If you choose to compress ZK announcements, you must do so for every node type
|
||||
druid.announcer.type=batch
|
||||
druid.curator.compress=true
|
||||
```
|
||||
|
||||
The historical module uses several of the default modules in [Configuration](Configuration.html) and has no uniques configs of its own.
|
|
@ -1,48 +1,12 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Historical
|
||||
=======
|
||||
Historical Node
|
||||
===============
|
||||
For Historical Node Configuration, see [Historial Configuration](Historical-Config.html).
|
||||
|
||||
Historical nodes load up historical segments and expose them for querying.
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=historical
|
||||
druid.port=8081
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.server.maxSize=10000000000
|
||||
|
||||
# Change these to make Druid faster
|
||||
druid.processing.buffer.sizeBytes=100000000
|
||||
druid.processing.numThreads=1
|
||||
|
||||
druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 10000000000}]
|
||||
```
|
||||
|
||||
Note: This will spin up a Historical node with the local filesystem as deep storage.
|
||||
|
||||
JVM Configuration
|
||||
-----------------
|
||||
The historical module uses several of the default modules in [Configuration](Configuration.html) and has no uniques configs of its own.
|
||||
|
||||
Running
|
||||
-------
|
||||
|
||||
|
|
|
@ -0,0 +1,249 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
For general Indexing Service information, see [here](Indexing-Service.html).
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
|
||||
```
|
||||
io.druid.cli.Main server overlord
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
-Ddruid.host=localhost
|
||||
-Ddruid.port=8080
|
||||
-Ddruid.service=overlord
|
||||
|
||||
-Ddruid.zk.service.host=localhost
|
||||
|
||||
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
|
||||
-Ddruid.db.connector.user=druid
|
||||
-Ddruid.db.connector.password=diurd
|
||||
|
||||
-Ddruid.selectors.indexing.serviceName=overlord
|
||||
-Ddruid.indexer.queue.startDelay=PT0M
|
||||
-Ddruid.indexer.runner.javaOpts="-server -Xmx1g"
|
||||
-Ddruid.indexer.runner.startPort=8081
|
||||
-Ddruid.indexer.fork.property.druid.computation.buffer.size=268435456
|
||||
```
|
||||
|
||||
Production Configs
|
||||
------------------
|
||||
These production configs are using S3 as a deep store and running the indexing service in distributed mode.
|
||||
|
||||
JVM settings for both overlord and middle manager:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx#{HEAP_MAX}g
|
||||
-Xms#{HEAP_MIN}g
|
||||
-XX:NewSize=#{NEW_SIZE}g
|
||||
-XX:MaxNewSize=#{MAX_NEW_SIZE}g
|
||||
-XX:+UseConcMarkSweepGC
|
||||
-XX:+PrintGCDetails
|
||||
-XX:+PrintGCTimeStamps
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
-Djava.io.tmpdir=/mnt/tmp
|
||||
|
||||
-Dcom.sun.management.jmxremote.port=17071
|
||||
-Dcom.sun.management.jmxremote.authenticate=false
|
||||
-Dcom.sun.management.jmxremote.ssl=false
|
||||
```
|
||||
|
||||
Runtime.properties for overlord:
|
||||
|
||||
```
|
||||
druid.host=#{IP_ADDR}:8080
|
||||
druid.port=8080
|
||||
druid.service=druid/prod/indexer
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.61"]
|
||||
|
||||
druid.zk.service.host=#{ZK_IPs}
|
||||
druid.zk.paths.base=/druid/prod
|
||||
|
||||
druid.discovery.curator.path=/prod/discovery
|
||||
|
||||
druid.s3.accessKey=#{ACCESS_KEY}
|
||||
druid.s3.secretKey=#{SECRET_KEY}
|
||||
|
||||
druid.db.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid
|
||||
druid.db.connector.user=#{MYSQL_USER}
|
||||
druid.db.connector.password=#{MYSQL_PW}
|
||||
druid.db.connector.useValidationQuery=true
|
||||
druid.db.tables.base=prod
|
||||
|
||||
druid.indexer.autoscale.doAutoscale=true
|
||||
druid.indexer.autoscale.strategy=ec2
|
||||
druid.indexer.autoscale.workerIdleTimeout=PT90m
|
||||
druid.indexer.autoscale.terminatePeriod=PT5M
|
||||
druid.indexer.autoscale.workerVersion=#{WORKER_VERSION}
|
||||
|
||||
druid.indexer.firehoseId.prefix=druid:prod:chat
|
||||
druid.indexer.logs.type=s3
|
||||
druid.indexer.logs.s3Bucket=#{INDEXER_LOGS_BUCKET}
|
||||
druid.indexer.logs.s3Prefix=prod/logs/v1
|
||||
druid.indexer.runner.type=remote
|
||||
druid.indexer.runner.compressZnodes=true
|
||||
druid.indexer.runner.minWorkerVersion=#{WORKER_VERSION}
|
||||
druid.indexer.storage.type=db
|
||||
|
||||
druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]
|
||||
|
||||
# Emit metrics over http
|
||||
druid.emitter=http
|
||||
druid.emitter.http.recipientBaseUrl=#{EMITTER_URL}
|
||||
|
||||
# If you choose to compress ZK announcements, you must do so for every node type
|
||||
druid.announcer.type=batch
|
||||
druid.curator.compress=true
|
||||
```
|
||||
|
||||
Runtime.properties for middle manager:
|
||||
|
||||
```
|
||||
druid.host=#{IP_ADDR}:8080
|
||||
druid.port=8080
|
||||
druid.service=druid/prod/worker
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.61","io.druid.extensions:druid-kafka-seven:0.6.61"]
|
||||
|
||||
druid.zk.service.host=#{ZK_IPs}
|
||||
druid.zk.paths.base=/druid/prod
|
||||
|
||||
druid.discovery.curator.path=/prod/discovery
|
||||
|
||||
druid.s3.accessKey=#{ACCESS_KEY}
|
||||
druid.s3.secretKey=#{SECRET_KEY}
|
||||
|
||||
druid.indexer.logs.type=s3
|
||||
druid.indexer.logs.s3Bucket=#{INDEXER_LOGS_BUCKET}
|
||||
druid.indexer.logs.s3Prefix=prod/logs/v1
|
||||
druid.indexer.runner.javaOpts=-server -Xmx#{HEAP_MAX}g -Xms#{HEAP_MIN}g -XX:NewSize=#{NEW_SIZE}m -XX:MaxNewSize=#{MAX_NEW_SIZE}6m -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
|
||||
druid.indexer.runner.startPort=8081
|
||||
druid.indexer.runner.taskDir=/mnt/persistent/task/
|
||||
druid.indexer.task.taskDir=/mnt/persistent/task/
|
||||
druid.indexer.task.chathandler.type=announce
|
||||
|
||||
druid.indexer.firehoseId.prefix=druid:prod:chat
|
||||
|
||||
druid.indexer.fork.property.druid.indexer.hadoopWorkingPath=/tmp/druid-indexing
|
||||
druid.indexer.fork.property.druid.computation.buffer.size=#{BUFFER_SIZE}
|
||||
druid.indexer.fork.property.druid.processing.numThreads=#{NUM_WORKER_THREADS}
|
||||
druid.indexer.fork.property.druid.request.logging.type=file
|
||||
druid.indexer.fork.property.druid.request.logging.dir=request_logs/
|
||||
druid.indexer.fork.property.druid.segmentCache.locations=[{"path": "/mnt/persistent/zk_druid", "maxSize": 0}]
|
||||
druid.indexer.fork.property.druid.storage.type=s3
|
||||
druid.indexer.fork.property.druid.storage.baseKey=prod/v1
|
||||
druid.indexer.fork.property.druid.storage.bucket=#{INDEXER_LOGS_BUCKET}
|
||||
druid.server.http.numThreads=20
|
||||
|
||||
druid.worker.capacity=#{NUM_WORKER_THREADS}
|
||||
druid.worker.ip=#{IP_ADDR}
|
||||
druid.worker.version=#{WORKER_VERSION}
|
||||
|
||||
druid.selectors.indexing.serviceName=druid:prod:indexer
|
||||
|
||||
druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]
|
||||
|
||||
# Emit metrics over http
|
||||
druid.emitter=http
|
||||
druid.emitter.http.recipientBaseUrl=#{EMITTER_URL}
|
||||
|
||||
# If you choose to compress ZK announcements, you must do so for every node type
|
||||
druid.announcer.type=batch
|
||||
druid.curator.compress=true
|
||||
```
|
||||
|
||||
#### Runtime Configuration
|
||||
|
||||
In addition to the configuration of some of the default modules in [Configuration](Configuration.html), the overlord has the following basic configs:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.runner.type`|Choices "local" or "remote". Indicates whether tasks should be run locally or in a distributed environment.|local|
|
||||
|`druid.indexer.storage.type`|Choices are "local" or "db". Indicates whether incoming tasks should be stored locally (in heap) or in a database. Storing incoming tasks in a database allows for tasks to be resumed if the overlord should fail.|local|
|
||||
|`druid.indexer.storage.recentlyFinishedThreshold`|A duration of time to store task results.|PT24H|
|
||||
|`druid.indexer.queue.maxSize`|Maximum number of active tasks at one time.|Integer.MAX_VALUE|
|
||||
|`druid.indexer.queue.startDelay`|Sleep this long before starting overlord queue management. This can be useful to give a cluster time to re-orient itself after e.g. a widespread network issue.|PT1M|
|
||||
|`druid.indexer.queue.restartDelay`|Sleep this long when overlord queue management throws an exception before trying again.|PT30S|
|
||||
|`druid.indexer.queue.storageSyncRate`|Sync overlord state this often with an underlying task persistence mechanism.|PT1M|
|
||||
|
||||
The following configs only apply if the overlord is running in remote mode:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.runner.taskAssignmentTimeout`|How long to wait after a task as been assigned to a middle manager before throwing an error.|PT5M|
|
||||
|`druid.indexer.runner.minWorkerVersion`|The minimum middle manager version to send tasks to. |none|
|
||||
|`druid.indexer.runner.compressZnodes`|Indicates whether or not the overlord should expect middle managers to compress Znodes.|false|
|
||||
|`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288|
|
||||
|
||||
There are additional configs for autoscaling (if it is enabled):
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.autoscale.strategy`|Choices are "noop" or "ec2". Sets the strategy to run when autoscaling is required.|noop|
|
||||
|`druid.indexer.autoscale.doAutoscale`|If set to "true" autoscaling will be enabled.|false|
|
||||
|`druid.indexer.autoscale.provisionPeriod`|How often to check whether or not new middle managers should be added.|PT1M|
|
||||
|`druid.indexer.autoscale.terminatePeriod`|How often to check when middle managers should be removed.|PT1H|
|
||||
|`druid.indexer.autoscale.originTime`|The starting reference timestamp that the terminate period increments upon.|2012-01-01T00:55:00.000Z|
|
||||
|`druid.indexer.autoscale.workerIdleTimeout`|How long can a worker be idle (not a run task) before it can be considered for termination.|PT10M|
|
||||
|`druid.indexer.autoscale.maxScalingDuration`|How long the overlord will wait around for a middle manager to show up before giving up.|PT15M|
|
||||
|`druid.indexer.autoscale.numEventsToTrack`|The number of autoscaling related events (node creation and termination) to track.|10|
|
||||
|`druid.indexer.autoscale.pendingTaskTimeout`|How long a task can be in "pending" state before the overlord tries to scale up.|PT30S|
|
||||
|`druid.indexer.autoscale.workerVersion`|If set, will only create nodes of set version during autoscaling. Overrides dynamic configuration. |null|
|
||||
|`druid.indexer.autoscale.workerPort`|The port that middle managers will run on.|8080|
|
||||
|
||||
#### Dynamic Configuration
|
||||
|
||||
Overlord dynamic configuration is mainly for autoscaling. The overlord reads a worker setup spec as a JSON object from the Druid [MySQL](MySQL.html) config table. This object contains information about the version of middle managers to create, the maximum and minimum number of middle managers in the cluster at one time, and additional information required to automatically create middle managers.
|
||||
|
||||
The JSON object can be submitted to the overlord via a POST request at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<port>/druid/indexer/v1/worker/setup
|
||||
```
|
||||
|
||||
A sample worker setup spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"minVersion":"some_version",
|
||||
"minNumWorkers":"0",
|
||||
"maxNumWorkers":"10",
|
||||
"nodeData": {
|
||||
"type":"ec2",
|
||||
"amiId":"ami-someId",
|
||||
"instanceType":"m1.xlarge",
|
||||
"minInstances":"1",
|
||||
"maxInstances":"1",
|
||||
"securityGroupIds":["securityGroupIds"],
|
||||
"keyName":"keyName"
|
||||
},
|
||||
"userData":{
|
||||
"classType":"galaxy",
|
||||
"env":"druid",
|
||||
"version":"druid_version",
|
||||
"type":"sample_cluster/worker"
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
Issuing a GET request at the same URL will return the current worker setup spec that is currently in place. The worker setup spec list above is just a sample and it is possible to extend the code base for other deployment environments. A description of the worker setup spec is shown below.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be the same as the coordinator version.|none|
|
||||
|`minNumWorkers`|The minimum number of workers that can be in the cluster at any given time.|0|
|
||||
|`maxNumWorkers`|The maximum number of workers that can be in the cluster at any given time.|0|
|
||||
|`nodeData`|A JSON object that contains metadata about new nodes to create.|none|
|
||||
|`userData`|A JSON object that contains metadata about how the node should register itself on startup. This data is sent with node creation requests.|none|
|
|
@ -1,7 +1,10 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Druid Indexing Service
|
||||
Indexing Service
|
||||
================
|
||||
For Indexing Service Configuration, see [Indexing Service Configuration](Indexing-Service-Config.html).
|
||||
|
||||
The indexing service is a highly-available, distributed service that runs indexing related tasks. Indexing service [tasks](Tasks.html) create (and sometimes destroy) Druid [segments](Segments.html). The indexing service has a master/slave like architecture.
|
||||
|
||||
The indexing service is composed of three main components: a peon component that can run a single task, a [Middle Manager](Middlemanager.html) component that manages peons, and an overlord component that manages task distribution to middle managers.
|
||||
|
@ -21,52 +24,14 @@ The truth is, the indexing service is an experience that is difficult to charact
|
|||
The indexing service is philosophical transcendence, an infallible truth that will shape your soul, mold your character, and define your reality. The indexing service is creating world peace, playing with puppies, unwrapping presents on Christmas morning, cradling a loved one, and beating Goro in Mortal Kombat for the first time. The indexing service is sustainable economic growth, global propensity, and a world of transparent financial transactions. The indexing service is a true belieber. The indexing service is panicking because you forgot you signed up for a course and the big exam is in a few minutes, only to wake up and realize it was all a dream. What is the indexing service? More like what isn’t the indexing service. The indexing service is here and it is ready, but are you?
|
||||
-->
|
||||
|
||||
|
||||
Overlord Node
|
||||
-----------------
|
||||
-------------
|
||||
|
||||
The overlord node is responsible for accepting tasks, coordinating task distribution, creating locks around tasks, and returning statuses to callers. Overlord can be configured to run in one of two modes - local or remote (local being default).
|
||||
In local mode overlord is also responsible for creating peons for executing tasks. When running the overlord in local mode, all middle manager and peon configurations must be provided as well.
|
||||
Local mode is typically used for simple workflows. In remote mode, the overlord and middle manager are run in separate processes and you can run each on a different server.
|
||||
This mode is recommended if you intend to use the indexing service as the single endpoint for all Druid indexing.
|
||||
|
||||
|
||||
Run Overlord
|
||||
----
|
||||
|
||||
```
|
||||
io.druid.cli.Main server overlord
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
-Ddruid.host=localhost
|
||||
-Ddruid.port=8080
|
||||
-Ddruid.service=overlord
|
||||
|
||||
-Ddruid.zk.service.host=localhost
|
||||
|
||||
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
|
||||
-Ddruid.db.connector.user=druid
|
||||
-Ddruid.db.connector.password=diurd
|
||||
|
||||
-Ddruid.selectors.indexing.serviceName=overlord
|
||||
-Ddruid.indexer.queue.startDelay=PT0M
|
||||
-Ddruid.indexer.runner.javaOpts="-server -Xmx1g"
|
||||
-Ddruid.indexer.runner.startPort=8081
|
||||
-Ddruid.indexer.fork.property.druid.computation.buffer.size=268435456
|
||||
```
|
||||
|
||||
You can now submit simple indexing tasks to the indexing service.
|
||||
|
||||
|
||||
|
||||
#### Submitting Tasks and Querying Task Status
|
||||
|
||||
Tasks are submitted to the overlord node in the form of JSON objects. Tasks can be submitted via POST requests to:
|
||||
|
@ -110,94 +75,8 @@ The Autoscaling mechanisms currently in place are tightly coupled with our deplo
|
|||
|
||||
If autoscaling is enabled, new middle managers may be added when a task has been in pending state for too long. Middle managers may be terminated if they have not run any tasks for a period of time.
|
||||
|
||||
#### JVM Configuration
|
||||
|
||||
In addition to the configuration of some of the default modules in [Configuration](Configuration.html), the overlord has the following basic configs:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.runner.type`|Choices "local" or "remote". Indicates whether tasks should be run locally or in a distributed environment.|local|
|
||||
|`druid.indexer.storage.type`|Choices are "local" or "db". Indicates whether incoming tasks should be stored locally (in heap) or in a database. Storing incoming tasks in a database allows for tasks to be resumed if the overlord should fail.|local|
|
||||
|`druid.indexer.storage.recentlyFinishedThreshold`|A duration of time to store task results.|PT24H|
|
||||
|`druid.indexer.queue.maxSize`|Maximum number of active tasks at one time.|Integer.MAX_VALUE|
|
||||
|`druid.indexer.queue.startDelay`|Sleep this long before starting overlord queue management. This can be useful to give a cluster time to re-orient itself after e.g. a widespread network issue.|PT1M|
|
||||
|`druid.indexer.queue.restartDelay`|Sleep this long when overlord queue management throws an exception before trying again.|PT30S|
|
||||
|`druid.indexer.queue.storageSyncRate`|Sync overlord state this often with an underlying task persistence mechanism.|PT1M|
|
||||
|
||||
The following configs only apply if the overlord is running in remote mode:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.runner.taskAssignmentTimeout`|How long to wait after a task as been assigned to a middle manager before throwing an error.|PT5M|
|
||||
|`druid.indexer.runner.minWorkerVersion`|The minimum middle manager version to send tasks to. |none|
|
||||
|`druid.indexer.runner.compressZnodes`|Indicates whether or not the overlord should expect middle managers to compress Znodes.|false|
|
||||
|`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288|
|
||||
|
||||
There are additional configs for autoscaling (if it is enabled):
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.autoscale.strategy`|Choices are "noop" or "ec2". Sets the strategy to run when autoscaling is required.|noop|
|
||||
|`druid.indexer.autoscale.doAutoscale`|If set to "true" autoscaling will be enabled.|false|
|
||||
|`druid.indexer.autoscale.provisionPeriod`|How often to check whether or not new middle managers should be added.|PT1M|
|
||||
|`druid.indexer.autoscale.terminatePeriod`|How often to check when middle managers should be removed.|PT1H|
|
||||
|`druid.indexer.autoscale.originTime`|The starting reference timestamp that the terminate period increments upon.|2012-01-01T00:55:00.000Z|
|
||||
|`druid.indexer.autoscale.workerIdleTimeout`|How long can a worker be idle (not a run task) before it can be considered for termination.|PT10M|
|
||||
|`druid.indexer.autoscale.maxScalingDuration`|How long the overlord will wait around for a middle manager to show up before giving up.|PT15M|
|
||||
|`druid.indexer.autoscale.numEventsToTrack`|The number of autoscaling related events (node creation and termination) to track.|10|
|
||||
|`druid.indexer.autoscale.pendingTaskTimeout`|How long a task can be in "pending" state before the overlord tries to scale up.|PT30S|
|
||||
|`druid.indexer.autoscale.workerVersion`|If set, will only create nodes of set version during autoscaling. Overrides dynamic configuration. |null|
|
||||
|`druid.indexer.autoscale.workerPort`|The port that middle managers will run on.|8080|
|
||||
|
||||
#### Dynamic Configuration
|
||||
|
||||
Overlord dynamic configuration is mainly for autoscaling. The overlord reads a worker setup spec as a JSON object from the Druid [MySQL](MySQL.html) config table. This object contains information about the version of middle managers to create, the maximum and minimum number of middle managers in the cluster at one time, and additional information required to automatically create middle managers.
|
||||
|
||||
The JSON object can be submitted to the overlord via a POST request at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<port>/druid/indexer/v1/worker/setup
|
||||
```
|
||||
|
||||
A sample worker setup spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"minVersion":"some_version",
|
||||
"minNumWorkers":"0",
|
||||
"maxNumWorkers":"10",
|
||||
"nodeData": {
|
||||
"type":"ec2",
|
||||
"amiId":"ami-someId",
|
||||
"instanceType":"m1.xlarge",
|
||||
"minInstances":"1",
|
||||
"maxInstances":"1",
|
||||
"securityGroupIds":["securityGroupIds"],
|
||||
"keyName":"keyName"
|
||||
},
|
||||
"userData":{
|
||||
"classType":"galaxy",
|
||||
"env":"druid",
|
||||
"version":"druid_version",
|
||||
"type":"sample_cluster/worker"
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
Issuing a GET request at the same URL will return the current worker setup spec that is currently in place. The worker setup spec list above is just a sample and it is possible to extend the code base for other deployment environments. A description of the worker setup spec is shown below.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be the same as the coordinator version.|none|
|
||||
|`minNumWorkers`|The minimum number of workers that can be in the cluster at any given time.|0|
|
||||
|`maxNumWorkers`|The maximum number of workers that can be in the cluster at any given time.|0|
|
||||
|`nodeData`|A JSON object that contains metadata about new nodes to create.|none|
|
||||
|`userData`|A JSON object that contains metadata about how the node should register itself on startup. This data is sent with node creation requests.|none|
|
||||
|
||||
|
||||
|
||||
Middle Managers
|
||||
-----
|
||||
---------------
|
||||
|
||||
See [Middle Manager](Middlemanager.html).
|
||||
|
||||
|
|
|
@ -0,0 +1,119 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Realtime Node Configuration
|
||||
===========================
|
||||
For general Real-time Node information, see [here](Realtime.html).
|
||||
|
||||
For Real-time Ingestion, see [Realtime Ingestion](Realtime-ingestion.html).
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server realtime
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=realtime
|
||||
druid.port=8083
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.61"]
|
||||
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
# The realtime config file.
|
||||
druid.realtime.specFile=/path/to/specFile
|
||||
|
||||
# Choices: db (hand off segments), noop (do not hand off segments).
|
||||
druid.publish.type=db
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.processing.buffer.sizeBytes=100000000
|
||||
```
|
||||
|
||||
Production Configs
|
||||
------------------
|
||||
These production configs are using S3 as a deep store.
|
||||
|
||||
JVM settings:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx#{HEAP_MAX}g
|
||||
-Xms#{HEAP_MIN}g
|
||||
-XX:NewSize=#{NEW_SIZE}g
|
||||
-XX:MaxNewSize=#{MAX_NEW_SIZE}g
|
||||
-XX:+UseConcMarkSweepGC
|
||||
-XX:+PrintGCDetails
|
||||
-XX:+PrintGCTimeStamps
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
-Djava.io.tmpdir=/mnt/tmp
|
||||
|
||||
-Dcom.sun.management.jmxremote.port=17071
|
||||
-Dcom.sun.management.jmxremote.authenticate=false
|
||||
-Dcom.sun.management.jmxremote.ssl=false
|
||||
```
|
||||
|
||||
Runtime.properties:
|
||||
|
||||
```
|
||||
druid.host=#{IP_ADDR}:8080
|
||||
druid.port=8080
|
||||
druid.service=druid/prod/realtime
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.61","io.druid.extensions:druid-kafka-seven:0.6.61"]
|
||||
|
||||
druid.zk.service.host=#{ZK_IPs}
|
||||
druid.zk.paths.base=/druid/prod
|
||||
|
||||
druid.s3.accessKey=#{ACCESS_KEY}
|
||||
druid.s3.secretKey=#{SECRET_KEY}
|
||||
|
||||
druid.db.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid
|
||||
druid.db.connector.user=#{MYSQL_USER}
|
||||
druid.db.connector.password=#{MYSQL_PW}
|
||||
druid.db.connector.useValidationQuery=true
|
||||
druid.db.tables.base=prod
|
||||
|
||||
druid.publish.type=db
|
||||
|
||||
druid.processing.numThreads=3
|
||||
|
||||
druid.request.logging.type=file
|
||||
druid.request.logging.dir=request_logs/
|
||||
|
||||
druid.realtime.specFile=conf/schemas.json
|
||||
|
||||
druid.segmentCache.locations=[{"path": "/mnt/persistent/zk_druid", "maxSize": 0}]
|
||||
|
||||
druid.storage.type=s3
|
||||
druid.storage.bucket=#{S3_STORAGE_BUCKET}
|
||||
druid.storage.baseKey=prod-realtime/v1
|
||||
|
||||
druid.monitoring.monitors=["com.metamx.metrics.SysMonitor", "io.druid.segment.realtime.RealtimeMetricsMonitor"]
|
||||
|
||||
# Emit metrics over http
|
||||
druid.emitter=http
|
||||
druid.emitter.http.recipientBaseUrl=#{EMITTER_URL}
|
||||
|
||||
# If you choose to compress ZK announcements, you must do so for every node type
|
||||
druid.announcer.type=batch
|
||||
druid.curator.compress=true
|
||||
```
|
||||
|
||||
The realtime module also uses several of the default modules in [Configuration](Configuration.html). For more information on the realtime spec file (or configuration file), see [realtime ingestion](Realtime-ingestion.html) page.
|
|
@ -2,22 +2,15 @@
|
|||
layout: doc_page
|
||||
---
|
||||
Realtime Data Ingestion
|
||||
========
|
||||
=======================
|
||||
For general Real-time Node information, see [here](Realtime.html).
|
||||
|
||||
Realtime data ingestion uses [Realtime nodes](Realtime.html) to index data and make it immediately available for querying. This data is periodically handed off (in the form of data segments) to [Historical](Historical.html) nodes, after which that data is forgotten by the Realtime nodes. This handoff, or "segment propagation," involves a series of interactions between various members of the Druid cluster. It is illustrated below.
|
||||
For Real-time Node Configuration, see [Realtime Configuration](Realtime-Config.html).
|
||||
|
||||
For writing your own plugins to the real-time node, see [Firehose](Firehose.html).
|
||||
|
||||
Much of the configuration governing Realtime nodes and the ingestion of data is set in the Realtime spec file, discussed on this page.
|
||||
|
||||
|
||||
Segment Propagation
|
||||
-------------------
|
||||
|
||||
The segment propagation diagram for real-time data ingestion can be seen below:
|
||||
|
||||
![Segment Propagation](../img/segmentPropagation.png "Segment Propagation")
|
||||
|
||||
You can read about the various components shown in this diagram under the Architecture section (see the menu on the left).
|
||||
|
||||
<a id="realtime-specfile"></a>
|
||||
## Realtime "specFile"
|
||||
|
||||
|
@ -108,14 +101,6 @@ This provides configuration for the data processing portion of the realtime stre
|
|||
|intermediatePersistPeriod|ISO8601 Period String|The period that determines the rate at which intermediate persists occur. These persists determine how often commits happen against the incoming realtime stream. If the realtime data loading process is interrupted at time T, it should be restarted to re-read data that arrived at T minus this period.|yes|
|
||||
|maxRowsInMemory|Number|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size.|yes|
|
||||
|
||||
### Firehose
|
||||
|
||||
See [Firehose](Firehose.html).
|
||||
|
||||
### Plumber
|
||||
|
||||
See [Plumber](Plumber.html)
|
||||
|
||||
Constraints
|
||||
-----------
|
||||
|
||||
|
@ -131,18 +116,4 @@ The following table summarizes constraints between settings in the spec file for
|
|||
|
||||
The normal, expected use cases have the following overall constraints: `indexGranularity < intermediatePersistPeriod =< windowPeriod < segmentGranularity`
|
||||
|
||||
If the RealtimeNode process runs out of heap, try adjusting druid.computation.buffer.size property which specifies a size in bytes that must fit into the heap.
|
||||
|
||||
|
||||
|
||||
Extending the code
|
||||
------------------
|
||||
|
||||
Realtime integration is intended to be extended in two ways:
|
||||
|
||||
1. Connect to data streams from varied systems ([Firehose](https://github.com/druid-io/druid-api/blob/master/src/main/java/io/druid/data/input/FirehoseFactory.java))
|
||||
2. Adjust the publishing strategy to match your needs ([Plumber](https://github.com/metamx/druid/blob/master/server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java))
|
||||
|
||||
The expectations are that the former will be very common and something that users of Druid will do on a fairly regular basis. Most users will probably never have to deal with the latter form of customization. Indeed, we hope that all potential use cases can be packaged up as part of Druid proper without requiring proprietary customization.
|
||||
|
||||
Given those expectations, adding a firehose is straightforward and completely encapsulated inside of the interface. Adding a plumber is more involved and requires understanding of how the system works to get right, it’s not impossible, but it’s not intended that individuals new to Druid will be able to do it immediately.
|
||||
If the Realtime Node process runs out of heap, try adjusting druid.computation.buffer.size property which specifies a size in bytes that must fit into the heap.
|
|
@ -1,54 +1,44 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Realtime Nodes
|
||||
========
|
||||
Real-time Node
|
||||
==============
|
||||
For Real-time Node Configuration, see [Realtime Configuration](Realtime-Config.html).
|
||||
|
||||
For Real-time Ingestion, see [Realtime Ingestion](Realtime-ingestion.html).
|
||||
|
||||
Realtime nodes provide a realtime index. Data indexed via these nodes is immediately available for querying. Realtime nodes will periodically build segments representing the data they’ve collected over some span of time and transfer these segments off to [Historical](Historical.html) nodes. They use ZooKeeper to monitor the transfer and MySQL to store metadata about the transfered segment. Once transfered, segments are forgotten by the Realtime nodes.
|
||||
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
### Running
|
||||
|
||||
```
|
||||
io.druid.cli.Main server realtime
|
||||
```
|
||||
Segment Propagation
|
||||
-------------------
|
||||
|
||||
With the following JVM configuration:
|
||||
The segment propagation diagram for real-time data ingestion can be seen below:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
![Segment Propagation](../img/segmentPropagation.png "Segment Propagation")
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=realtime
|
||||
druid.port=8083
|
||||
You can read about the various components shown in this diagram under the Architecture section (see the menu on the left).
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.52"]
|
||||
### Firehose
|
||||
|
||||
See [Firehose](Firehose.html).
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
### Plumber
|
||||
|
||||
# The realtime config file.
|
||||
druid.realtime.specFile=/path/to/specFile
|
||||
See [Plumber](Plumber.html)
|
||||
|
||||
# Choices: db (hand off segments), noop (do not hand off segments).
|
||||
druid.publish.type=db
|
||||
Extending the code
|
||||
------------------
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
Realtime integration is intended to be extended in two ways:
|
||||
|
||||
druid.processing.buffer.sizeBytes=100000000
|
||||
```
|
||||
1. Connect to data streams from varied systems ([Firehose](https://github.com/druid-io/druid-api/blob/master/src/main/java/io/druid/data/input/FirehoseFactory.java))
|
||||
2. Adjust the publishing strategy to match your needs ([Plumber](https://github.com/metamx/druid/blob/master/server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java))
|
||||
|
||||
The realtime module also uses several of the default modules in [Configuration](Configuration.html). For more information on the realtime spec file (or configuration file), see [realtime ingestion](Realtime-ingestion.html) page.
|
||||
The expectations are that the former will be very common and something that users of Druid will do on a fairly regular basis. Most users will probably never have to deal with the latter form of customization. Indeed, we hope that all potential use cases can be packaged up as part of Druid proper without requiring proprietary customization.
|
||||
|
||||
|
||||
Requirements
|
||||
------------
|
||||
|
||||
Realtime nodes currently require a Kafka cluster to sit in front of them and collect results. There’s [more configuration](Tutorial\:-Loading-Your-Data-Part-2.md#set-up-kafka) required for these as well.
|
||||
Given those expectations, adding a firehose is straightforward and completely encapsulated inside of the interface. Adding a plumber is more involved and requires understanding of how the system works to get right, it’s not impossible, but it’s not intended that individuals new to Druid will be able to do it immediately.
|
||||
|
|
|
@ -52,7 +52,7 @@ There are 7 main parts to a timeseries query:
|
|||
|granularity|Defines the granularity of the query. See [Granularities](Granularities.html)|yes|
|
||||
|filter|See [Filters](Filters.html)|no|
|
||||
|aggregations|See [Aggregations](Aggregations.html)|yes|
|
||||
|postAggregations|See [Post Aggregations](Post-Aggregations.html)|no|
|
||||
|postAggregations|See [Post Aggregations](Post-aggregations.html)|no|
|
||||
|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
|
||||
|context|An additional JSON Object which can be used to specify certain flags.|no|
|
||||
|
||||
|
@ -69,4 +69,4 @@ To pull it all together, the above query would return 2 data points, one for eac
|
|||
"result": { "sample_name1": <some_value>, "sample_name2": <some_value>, "sample_divide": <some_value> }
|
||||
}
|
||||
]
|
||||
```
|
||||
```
|
||||
|
|
|
@ -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.52-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.61-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.52
|
||||
cd druid-services-0.6.61
|
||||
```
|
||||
|
||||
You should see a bunch of files:
|
||||
|
@ -323,7 +323,7 @@ Feel free to tweak other query parameters to answer other questions you may have
|
|||
Next Steps
|
||||
----------
|
||||
|
||||
What to know even more information about the Druid Cluster? Check out [The Druid Cluster](Tutorial%3A-The-Druid-Cluster.html).
|
||||
Want to know even more information about the Druid Cluster? Check out [The Druid Cluster](Tutorial%3A-The-Druid-Cluster.html).
|
||||
|
||||
Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-1.html).
|
||||
|
||||
|
|
|
@ -45,7 +45,7 @@ With real-world data, we recommend having a message bus such as [Apache Kafka](h
|
|||
<a id="set-up-kafka"></a>
|
||||
#### Setting up Kafka
|
||||
|
||||
[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.52/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node.
|
||||
[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.61/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node.
|
||||
|
||||
Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html).
|
||||
|
||||
|
|
|
@ -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.52-bin.tar.gz)
|
||||
You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.61-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.52"]
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.61"]
|
||||
|
||||
# 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.52","io.druid.extensions:druid-kafka-seven:0.6.52"]
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.61","io.druid.extensions:druid-kafka-seven:0.6.61"]
|
||||
|
||||
# Change this config to db to hand off to the rest of the Druid cluster
|
||||
druid.publish.type=noop
|
||||
|
|
|
@ -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.52-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.61-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.52
|
||||
cd druid-services-0.6.61
|
||||
```
|
||||
|
||||
You should see a bunch of files:
|
||||
|
|
|
@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source.
|
|||
|
||||
h3. Download a Tarball
|
||||
|
||||
We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.52-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.61-bin.tar.gz.
|
||||
Download this bad boy to a directory of your choosing.
|
||||
|
||||
You can extract the awesomeness within by issuing:
|
||||
|
|
|
@ -14,8 +14,15 @@ h2. Getting Started
|
|||
* "Tutorial: Loading Your Data Part 2":./Tutorial:-Loading-Your-Data-Part-2.html
|
||||
* "Tutorial: All About Queries":./Tutorial:-All-About-Queries.html
|
||||
|
||||
h2. Configuration
|
||||
* "Common Configuration":Configuration.html
|
||||
* "Realtime":Realtime-Config.html
|
||||
* "Coordinator":Coordinator-Config.html
|
||||
* "Historical":Historical-Config.html
|
||||
* "Broker":Broker-Config.html
|
||||
* "Indexing Service":Indexing-Service-Config.html
|
||||
|
||||
h2. Operations
|
||||
* "Configuration":Configuration.html
|
||||
* "Extending Druid":./Modules.html
|
||||
* "Cluster Setup":./Cluster-setup.html
|
||||
* "Booting a Production Cluster":./Booting-a-production-cluster.html
|
||||
|
@ -23,8 +30,6 @@ h2. Operations
|
|||
|
||||
h2. Data Ingestion
|
||||
* "Realtime":./Realtime-ingestion.html
|
||||
** "Firehose":./Firehose.html
|
||||
** "Plumber":./Plumber.html
|
||||
* "Batch":./Batch-ingestion.html
|
||||
* "Indexing Service":./Indexing-Service.html
|
||||
** "Tasks":./Tasks.html
|
||||
|
|
|
@ -4,7 +4,7 @@ druid.port=8081
|
|||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.52"]
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.61"]
|
||||
|
||||
# Dummy read only AWS account (used to download example data)
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
|
|
|
@ -13,3 +13,5 @@ druid.indexer.queue.startDelay=PT0M
|
|||
druid.indexer.runner.javaOpts="-server -Xmx1g"
|
||||
druid.indexer.runner.startPort=8088
|
||||
druid.indexer.fork.property.druid.computation.buffer.size=268435456
|
||||
druid.indexer.fork.property.druid.processing.numThreads=1
|
||||
|
||||
|
|
|
@ -4,7 +4,7 @@ druid.port=8083
|
|||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.52","io.druid.extensions:druid-kafka-seven:0.6.52","io.druid.extensions:druid-rabbitmq:0.6.52"]
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.61","io.druid.extensions:druid-kafka-seven:0.6.61","io.druid.extensions:druid-rabbitmq:0.6.61"]
|
||||
|
||||
# Change this config to db to hand off to the rest of the Druid cluster
|
||||
druid.publish.type=noop
|
||||
|
@ -15,3 +15,4 @@ druid.publish.type=noop
|
|||
# druid.db.connector.password=diurd
|
||||
|
||||
druid.processing.buffer.sizeBytes=100000000
|
||||
druid.processing.numThreads=1
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.Module;
|
|||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.inject.Binder;
|
||||
import io.druid.examples.flights.FlightsFirehoseFactory;
|
||||
import io.druid.examples.rand.RandomFirehoseFactory;
|
||||
import io.druid.examples.twitter.TwitterSpritzerFirehoseFactory;
|
||||
import io.druid.examples.web.WebFirehoseFactory;
|
||||
|
@ -43,7 +42,6 @@ public class ExamplesDruidModule implements DruidModule
|
|||
new SimpleModule("ExamplesModule")
|
||||
.registerSubtypes(
|
||||
new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"),
|
||||
new NamedType(FlightsFirehoseFactory.class, "flights"),
|
||||
new NamedType(RandomFirehoseFactory.class, "rand"),
|
||||
new NamedType(WebFirehoseFactory.class, "webstream")
|
||||
)
|
||||
|
|
|
@ -1,133 +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.examples.flights;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.parsers.CSVParser;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.Period;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.FileWriter;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FlightsConverter
|
||||
{
|
||||
|
||||
private static final String[] METRIC_DIMENSIONS = new String[]{
|
||||
"Distance",
|
||||
"TaxiIn",
|
||||
"TaxiOut",
|
||||
"CarrierDelay",
|
||||
"WeatherDelay",
|
||||
"NASDelay",
|
||||
"SecurityDelay",
|
||||
"LateAircraftDelay",
|
||||
"ArrDelay",
|
||||
"DepDelay",
|
||||
"CRSElapsedTime",
|
||||
"ActualElapsedTime",
|
||||
"AirTime"
|
||||
};
|
||||
|
||||
public static void main(String[] args) throws IOException
|
||||
{
|
||||
DateTimeZone.setDefault(DateTimeZone.UTC);
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
||||
File flightsDataDirectory = new File(args[0]);
|
||||
File flightsOutputDirectory = new File(args[1]);
|
||||
flightsOutputDirectory.mkdirs();
|
||||
|
||||
for (File flightsDataFile : flightsDataDirectory.listFiles()) {
|
||||
System.out.printf("Processing file[%s]%n", flightsDataFile);
|
||||
|
||||
CSVParser parser = new CSVParser();
|
||||
BufferedReader in = null;
|
||||
BufferedWriter out = null;
|
||||
|
||||
try {
|
||||
in = new BufferedReader(new FileReader(flightsDataFile));
|
||||
out = new BufferedWriter(
|
||||
new FileWriter(
|
||||
new File(
|
||||
flightsOutputDirectory,
|
||||
flightsDataFile.getName().replace("csv", "json")
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
int count = 0;
|
||||
long time = System.currentTimeMillis();
|
||||
parser.setFieldNames(in.readLine());
|
||||
String line = null;
|
||||
while ((line = in.readLine()) != null) {
|
||||
if (++count % 100000 == 0) {
|
||||
System.out.printf(
|
||||
"File[%s], processed %,d lines in %,d millis.%n",
|
||||
flightsDataFile.getName(), count, System.currentTimeMillis() - time
|
||||
);
|
||||
time = System.currentTimeMillis();
|
||||
}
|
||||
Map<String, Object> event = parser.parse(line);
|
||||
|
||||
int year = Integer.parseInt(event.get("Year").toString());
|
||||
int month = Integer.parseInt(event.get("Month").toString());
|
||||
int dayOfMonth = Integer.parseInt(event.get("DayofMonth").toString());
|
||||
int departureTime = Integer.parseInt(event.get("CRSDepTime").toString());
|
||||
int hourOfDay = departureTime / 100;
|
||||
final int minuteOfHour = departureTime % 100;
|
||||
|
||||
DateTime timestamp = new DateTime(String.format("%4d-%02d-%02d", year, month, dayOfMonth))
|
||||
.plus(new Period(hourOfDay, minuteOfHour, 0, 0));
|
||||
|
||||
event.put("timestamp", timestamp);
|
||||
|
||||
for (String metricDimension : METRIC_DIMENSIONS) {
|
||||
String value = event.get(metricDimension).toString();
|
||||
|
||||
if (value.equals("NA")) {
|
||||
event.put(metricDimension, 0);
|
||||
}
|
||||
else {
|
||||
event.put(metricDimension, Integer.parseInt(value));
|
||||
}
|
||||
}
|
||||
|
||||
out.write(mapper.writeValueAsString(event));
|
||||
out.write("\n");
|
||||
}
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(in);
|
||||
Closeables.closeQuietly(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,146 +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.examples.flights;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.io.Closeables;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.StringInputRowParser;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.Iterator;
|
||||
import java.util.zip.GZIPInputStream;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FlightsFirehoseFactory implements FirehoseFactory
|
||||
{
|
||||
private final String flightsFileLocation;
|
||||
private final StringInputRowParser parser;
|
||||
|
||||
@JsonCreator
|
||||
public FlightsFirehoseFactory(
|
||||
@JsonProperty("directory") String flightsFilesDirectory,
|
||||
@JsonProperty("parser") StringInputRowParser parser
|
||||
)
|
||||
{
|
||||
this.flightsFileLocation = flightsFilesDirectory;
|
||||
this.parser = parser;
|
||||
}
|
||||
|
||||
@JsonProperty("directory")
|
||||
public String getFlightsFileLocation()
|
||||
{
|
||||
return flightsFileLocation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
{
|
||||
File dir = new File(flightsFileLocation);
|
||||
|
||||
final Iterator<File> files = Iterators.forArray(dir.listFiles());
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
BufferedReader in = null;
|
||||
String line = null;
|
||||
|
||||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
try {
|
||||
if (line != null) {
|
||||
return true;
|
||||
}
|
||||
else if (in != null) {
|
||||
line = in.readLine();
|
||||
|
||||
if (line == null) {
|
||||
Closeables.closeQuietly(in);
|
||||
in = null;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (files.hasNext()) {
|
||||
final File nextFile = files.next();
|
||||
|
||||
if (nextFile.getName().endsWith(".gz")) {
|
||||
in = new BufferedReader(
|
||||
new InputStreamReader(new GZIPInputStream(new FileInputStream(nextFile)), Charsets.UTF_8)
|
||||
);
|
||||
}
|
||||
else {
|
||||
in = new BufferedReader(new FileReader(nextFile));
|
||||
}
|
||||
return hasMore();
|
||||
}
|
||||
Thread.currentThread().join();
|
||||
return false;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRow nextRow()
|
||||
{
|
||||
final InputRow retVal = parser.parse(line);
|
||||
line = null;
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
return new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
Closeables.closeQuietly(in);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -28,6 +28,7 @@ import io.druid.data.input.Firehose;
|
|||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.LinkedList;
|
||||
|
@ -54,81 +55,89 @@ import static java.lang.Thread.sleep;
|
|||
* the moment an event is delivered.)
|
||||
* Values are offset by adding the modulus of the token number to the random number
|
||||
* so that token values have distinct, non-overlapping ranges.
|
||||
*
|
||||
* <p/>
|
||||
* </p>
|
||||
* Example spec file:
|
||||
* <pre>
|
||||
[{
|
||||
"schema" : { "dataSource":"randseq",
|
||||
"aggregators":[ {"type":"count", "name":"events"},
|
||||
{"type":"doubleSum","name":"outColumn","fieldName":"inColumn"} ],
|
||||
"indexGranularity":"minute",
|
||||
"shardSpec" : { "type": "none" } },
|
||||
"config" : { "maxRowsInMemory" : 50000,
|
||||
"intermediatePersistPeriod" : "PT2m" },
|
||||
|
||||
"firehose" : { "type" : "rand",
|
||||
"sleepUsec": 100000,
|
||||
"maxGeneratedRows" : 5000000,
|
||||
"seed" : 0,
|
||||
"nTokens" : 19,
|
||||
"nPerSleep" : 3
|
||||
},
|
||||
|
||||
"plumber" : { "type" : "realtime",
|
||||
"windowPeriod" : "PT5m",
|
||||
"segmentGranularity":"hour",
|
||||
"basePersistDirectory" : "/tmp/realtime/basePersist" }
|
||||
}]
|
||||
* </pre>
|
||||
* [{
|
||||
* "schema" : { "dataSource":"randseq",
|
||||
* "aggregators":[ {"type":"count", "name":"events"},
|
||||
* {"type":"doubleSum","name":"outColumn","fieldName":"inColumn"} ],
|
||||
* "indexGranularity":"minute",
|
||||
* "shardSpec" : { "type": "none" } },
|
||||
* "config" : { "maxRowsInMemory" : 50000,
|
||||
* "intermediatePersistPeriod" : "PT2m" },
|
||||
*
|
||||
* "firehose" : { "type" : "rand",
|
||||
* "sleepUsec": 100000,
|
||||
* "maxGeneratedRows" : 5000000,
|
||||
* "seed" : 0,
|
||||
* "nTokens" : 19,
|
||||
* "nPerSleep" : 3
|
||||
* },
|
||||
*
|
||||
* "plumber" : { "type" : "realtime",
|
||||
* "windowPeriod" : "PT5m",
|
||||
* "segmentGranularity":"hour",
|
||||
* "basePersistDirectory" : "/tmp/realtime/basePersist" }
|
||||
* }]
|
||||
* </pre>
|
||||
* <p/>
|
||||
* Example query using POST to /druid/v2/ (where UTC date and time MUST include the current hour):
|
||||
* <pre>
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "randSeq",
|
||||
"granularity": "all",
|
||||
"dimensions": [],
|
||||
"aggregations":[
|
||||
{ "type": "count", "name": "rows"},
|
||||
{ "type": "doubleSum", "fieldName": "events", "name": "e"},
|
||||
{ "type": "doubleSum", "fieldName": "outColumn", "name": "randomNumberSum"}
|
||||
],
|
||||
"postAggregations":[
|
||||
{ "type":"arithmetic",
|
||||
"name":"avg_random",
|
||||
"fn":"/",
|
||||
"fields":[ {"type":"fieldAccess","name":"randomNumberSum","fieldName":"randomNumberSum"},
|
||||
{"type":"fieldAccess","name":"rows","fieldName":"rows"} ]}
|
||||
],
|
||||
"intervals":["2012-10-01T00:00/2020-01-01T00"]
|
||||
}
|
||||
* {
|
||||
* "queryType": "groupBy",
|
||||
* "dataSource": "randSeq",
|
||||
* "granularity": "all",
|
||||
* "dimensions": [],
|
||||
* "aggregations":[
|
||||
* { "type": "count", "name": "rows"},
|
||||
* { "type": "doubleSum", "fieldName": "events", "name": "e"},
|
||||
* { "type": "doubleSum", "fieldName": "outColumn", "name": "randomNumberSum"}
|
||||
* ],
|
||||
* "postAggregations":[
|
||||
* { "type":"arithmetic",
|
||||
* "name":"avg_random",
|
||||
* "fn":"/",
|
||||
* "fields":[ {"type":"fieldAccess","name":"randomNumberSum","fieldName":"randomNumberSum"},
|
||||
* {"type":"fieldAccess","name":"rows","fieldName":"rows"} ]}
|
||||
* ],
|
||||
* "intervals":["2012-10-01T00:00/2020-01-01T00"]
|
||||
* }
|
||||
* </pre>
|
||||
*/
|
||||
@JsonTypeName("rand")
|
||||
public class RandomFirehoseFactory implements FirehoseFactory
|
||||
public class RandomFirehoseFactory implements FirehoseFactory<InputRowParser>
|
||||
{
|
||||
private static final Logger log = new Logger(RandomFirehoseFactory.class);
|
||||
/** msec to sleep before generating a new row; if this and delayNsec are 0, then go as fast as possible.
|
||||
/**
|
||||
* msec to sleep before generating a new row; if this and delayNsec are 0, then go as fast as possible.
|
||||
* json param sleepUsec (microseconds) is used to initialize this.
|
||||
*/
|
||||
private final long delayMsec;
|
||||
/** nsec to sleep before generating a new row; if this and delayMsec are 0, then go as fast as possible.
|
||||
/**
|
||||
* nsec to sleep before generating a new row; if this and delayMsec are 0, then go as fast as possible.
|
||||
* json param sleepUsec (microseconds) is used to initialize this.
|
||||
*/
|
||||
*/
|
||||
private final int delayNsec;
|
||||
/** max rows to generate, -1 is infinite, 0 means nothing is generated; use this to prevent
|
||||
* infinite space consumption or to see what happens when a Firehose stops delivering
|
||||
* values, or to have hasMore() return false.
|
||||
/**
|
||||
* max rows to generate, -1 is infinite, 0 means nothing is generated; use this to prevent
|
||||
* infinite space consumption or to see what happens when a Firehose stops delivering
|
||||
* values, or to have hasMore() return false.
|
||||
*/
|
||||
private final long maxGeneratedRows;
|
||||
/** seed for random number generator; if 0, then no seed is used. */
|
||||
/**
|
||||
* seed for random number generator; if 0, then no seed is used.
|
||||
*/
|
||||
private final long seed;
|
||||
/** number of tokens to randomly associate with values (no heap limits). This can be used to
|
||||
/**
|
||||
* number of tokens to randomly associate with values (no heap limits). This can be used to
|
||||
* stress test the number of tokens.
|
||||
*/
|
||||
private final int nTokens;
|
||||
/** Number of token events per sleep interval. */
|
||||
/**
|
||||
* Number of token events per sleep interval.
|
||||
*/
|
||||
private final int nPerSleep;
|
||||
|
||||
@JsonCreator
|
||||
|
@ -143,29 +152,30 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
|||
long nsec = (sleepUsec > 0) ? sleepUsec * 1000L : 0;
|
||||
long msec = nsec / 1000000L;
|
||||
this.delayMsec = msec;
|
||||
this.delayNsec = (int)(nsec - (msec * 1000000L));
|
||||
this.delayNsec = (int) (nsec - (msec * 1000000L));
|
||||
this.maxGeneratedRows = maxGeneratedRows;
|
||||
this.seed = seed;
|
||||
this.nTokens = nTokens;
|
||||
this.nPerSleep = nPerSleep;
|
||||
if (nTokens <= 0) {
|
||||
log.warn("nTokens parameter " + nTokens +" ignored; must be greater than or equal to 1");
|
||||
log.warn("nTokens parameter " + nTokens + " ignored; must be greater than or equal to 1");
|
||||
nTokens = 1;
|
||||
}
|
||||
if (nPerSleep <= 0) {
|
||||
log.warn("nPerSleep parameter " + nPerSleep +" ignored; must be greater than or equal to 1");
|
||||
log.warn("nPerSleep parameter " + nPerSleep + " ignored; must be greater than or equal to 1");
|
||||
nPerSleep = 1;
|
||||
}
|
||||
log.info("maxGeneratedRows=" + maxGeneratedRows);
|
||||
log.info("seed=" + ( (seed == 0L) ? "random value" : seed ));
|
||||
log.info("seed=" + ((seed == 0L) ? "random value" : seed));
|
||||
log.info("nTokens=" + nTokens);
|
||||
log.info("nPerSleep=" + nPerSleep);
|
||||
double dmsec = (double)delayMsec + ((double)this.delayNsec)/1000000.;
|
||||
double dmsec = (double) delayMsec + ((double) this.delayNsec) / 1000000.;
|
||||
if (dmsec > 0.0) {
|
||||
log.info("sleep period=" + dmsec + "msec");
|
||||
log.info("approximate max rate of record generation=" + (nPerSleep * 1000./dmsec) + "/sec" +
|
||||
" or " + (60. * nPerSleep * 1000./dmsec) + "/minute"
|
||||
);
|
||||
log.info(
|
||||
"approximate max rate of record generation=" + (nPerSleep * 1000. / dmsec) + "/sec" +
|
||||
" or " + (60. * nPerSleep * 1000. / dmsec) + "/minute"
|
||||
);
|
||||
} else {
|
||||
log.info("sleep period= NONE");
|
||||
log.info("approximate max rate of record generation= as fast as possible");
|
||||
|
@ -173,7 +183,7 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(InputRowParser parser) throws IOException
|
||||
{
|
||||
final LinkedList<String> dimensions = new LinkedList<String>();
|
||||
dimensions.add("inColumn");
|
||||
|
@ -189,7 +199,7 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
|||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
if (maxGeneratedRows >= 0 && rowCount >= maxGeneratedRows) {
|
||||
if (maxGeneratedRows >= 0 && rowCount >= maxGeneratedRows) {
|
||||
return waitIfmaxGeneratedRows;
|
||||
} else {
|
||||
return true; // there are always more random numbers
|
||||
|
@ -203,7 +213,7 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
|||
final long nth = (rowCount % nTokens) + 1;
|
||||
long sleepMsec = delayMsec;
|
||||
// all done?
|
||||
if (maxGeneratedRows >= 0 && rowCount >= maxGeneratedRows && waitIfmaxGeneratedRows) {
|
||||
if (maxGeneratedRows >= 0 && rowCount >= maxGeneratedRows && waitIfmaxGeneratedRows) {
|
||||
// sleep a long time instead of terminating
|
||||
sleepMsec = 2000000000L;
|
||||
}
|
||||
|
@ -212,7 +222,8 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
|||
if (modulus == 0) {
|
||||
sleep(sleepMsec, delayNsec);
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException("InterruptedException");
|
||||
}
|
||||
}
|
||||
|
@ -221,7 +232,7 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
final Map<String, Object> theMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||
theMap.put("inColumn", anotherRand((int)nth));
|
||||
theMap.put("inColumn", anotherRand((int) nth));
|
||||
theMap.put("target", ("a" + nth));
|
||||
return new MapBasedInputRow(System.currentTimeMillis(), dimensions, theMap);
|
||||
}
|
||||
|
@ -229,7 +240,7 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
|||
private Float anotherRand(int scale)
|
||||
{
|
||||
double f = rand.nextDouble(); // [0.0,1.0]
|
||||
return new Float(f + (double)scale);
|
||||
return new Float(f + (double) scale);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -253,4 +264,10 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser getParser()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import io.druid.data.input.Firehose;
|
|||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import twitter4j.ConnectionLifeCycleListener;
|
||||
import twitter4j.HashtagEntity;
|
||||
import twitter4j.StallWarning;
|
||||
|
@ -82,7 +83,7 @@ import static java.lang.Thread.sleep;
|
|||
* @author pbaclace
|
||||
*/
|
||||
@JsonTypeName("twitzer")
|
||||
public class TwitterSpritzerFirehoseFactory implements FirehoseFactory {
|
||||
public class TwitterSpritzerFirehoseFactory implements FirehoseFactory<InputRowParser> {
|
||||
private static final Logger log = new Logger(TwitterSpritzerFirehoseFactory.class);
|
||||
/**
|
||||
* max events to receive, -1 is infinite, 0 means nothing is delivered; use this to prevent
|
||||
|
@ -111,7 +112,7 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(InputRowParser parser) throws IOException
|
||||
{
|
||||
final ConnectionLifeCycleListener connectionLifeCycleListener = new ConnectionLifeCycleListener() {
|
||||
@Override
|
||||
|
@ -316,4 +317,10 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser getParser()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import io.druid.data.input.Firehose;
|
|||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.utils.Runnables;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
|
@ -38,9 +39,10 @@ import java.util.Map;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@JsonTypeName("webstream")
|
||||
public class WebFirehoseFactory implements FirehoseFactory
|
||||
public class WebFirehoseFactory implements FirehoseFactory<InputRowParser>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(WebFirehoseFactory.class);
|
||||
|
||||
private final String timeFormat;
|
||||
private final UpdateStreamFactory factory;
|
||||
private final long queueWaitTime = 15L;
|
||||
|
@ -72,7 +74,7 @@ public class WebFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(InputRowParser parser) throws IOException
|
||||
{
|
||||
|
||||
final UpdateStream updateStream = factory.build();
|
||||
|
@ -131,4 +133,10 @@ public class WebFirehoseFactory implements FirehoseFactory
|
|||
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser getParser()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -75,7 +75,7 @@ public class WebFirehoseFactoryTest
|
|||
public void testDimensions() throws Exception
|
||||
{
|
||||
InputRow inputRow;
|
||||
Firehose firehose = webbie.connect();
|
||||
Firehose firehose = webbie.connect(null);
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
|
@ -90,7 +90,7 @@ public class WebFirehoseFactoryTest
|
|||
public void testPosixTimeStamp() throws Exception
|
||||
{
|
||||
InputRow inputRow;
|
||||
Firehose firehose = webbie.connect();
|
||||
Firehose firehose = webbie.connect(null);
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
|
@ -114,7 +114,7 @@ public class WebFirehoseFactoryTest
|
|||
},
|
||||
"auto"
|
||||
);
|
||||
Firehose firehose1 = webbie3.connect();
|
||||
Firehose firehose1 = webbie3.connect(null);
|
||||
if (firehose1.hasMore()) {
|
||||
long milliSeconds = firehose1.nextRow().getTimestampFromEpoch();
|
||||
DateTime date = new DateTime("2013-07-08");
|
||||
|
@ -138,7 +138,7 @@ public class WebFirehoseFactoryTest
|
|||
},
|
||||
null
|
||||
);
|
||||
Firehose firehose2 = webbie2.connect();
|
||||
Firehose firehose2 = webbie2.connect(null);
|
||||
if (firehose2.hasMore()) {
|
||||
long milliSeconds = firehose2.nextRow().getTimestampFromEpoch();
|
||||
DateTime date = new DateTime("2013-07-08");
|
||||
|
@ -151,7 +151,7 @@ public class WebFirehoseFactoryTest
|
|||
@Test
|
||||
public void testAutoMilliSecondsTimeStamp() throws Exception
|
||||
{
|
||||
Firehose firehose3 = webbie1.connect();
|
||||
Firehose firehose3 = webbie1.connect(null);
|
||||
if (firehose3.hasMore()) {
|
||||
long milliSeconds = firehose3.nextRow().getTimestampFromEpoch();
|
||||
DateTime date = new DateTime("2013-07-08");
|
||||
|
@ -165,7 +165,7 @@ public class WebFirehoseFactoryTest
|
|||
public void testGetDimension() throws Exception
|
||||
{
|
||||
InputRow inputRow;
|
||||
Firehose firehose = webbie1.connect();
|
||||
Firehose firehose = webbie1.connect(null);
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
|
@ -181,7 +181,7 @@ public class WebFirehoseFactoryTest
|
|||
public void testGetFloatMetric() throws Exception
|
||||
{
|
||||
InputRow inputRow;
|
||||
Firehose firehose = webbie1.connect();
|
||||
Firehose firehose = webbie1.connect(null);
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
@ -97,6 +97,11 @@
|
|||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.clearspring.analytics</groupId>
|
||||
<artifactId>stream</artifactId>
|
||||
<version>2.5.2</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
|
|
@ -0,0 +1,357 @@
|
|||
/*
|
||||
* 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.indexer;
|
||||
|
||||
import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
|
||||
import com.clearspring.analytics.stream.cardinality.HyperLogLog;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.hash.HashFunction;
|
||||
import com.google.common.hash.Hashing;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.Rows;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.indexer.granularity.UniformGranularitySpec;
|
||||
import io.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
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.Reducer;
|
||||
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
|
||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeComparator;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Determines appropriate ShardSpecs for a job by determining approximate cardinality of data set using HyperLogLog
|
||||
*/
|
||||
public class DetermineHashedPartitionsJob implements Jobby
|
||||
{
|
||||
private static final int MAX_SHARDS = 128;
|
||||
private static final Logger log = new Logger(DetermineHashedPartitionsJob.class);
|
||||
private static final int HYPER_LOG_LOG_BIT_SIZE = 20;
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
|
||||
public DetermineHashedPartitionsJob(
|
||||
HadoopDruidIndexerConfig config
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
public boolean run()
|
||||
{
|
||||
try {
|
||||
/*
|
||||
* Group by (timestamp, dimensions) so we can correctly count dimension values as they would appear
|
||||
* in the final segment.
|
||||
*/
|
||||
long startTime = System.currentTimeMillis();
|
||||
final Job groupByJob = new Job(
|
||||
new Configuration(),
|
||||
String.format("%s-determine_partitions_hashed-%s", config.getDataSource(), config.getIntervals())
|
||||
);
|
||||
|
||||
JobHelper.injectSystemProperties(groupByJob);
|
||||
groupByJob.setInputFormatClass(TextInputFormat.class);
|
||||
groupByJob.setMapperClass(DetermineCardinalityMapper.class);
|
||||
groupByJob.setMapOutputKeyClass(LongWritable.class);
|
||||
groupByJob.setMapOutputValueClass(BytesWritable.class);
|
||||
groupByJob.setReducerClass(DetermineCardinalityReducer.class);
|
||||
groupByJob.setOutputKeyClass(NullWritable.class);
|
||||
groupByJob.setOutputValueClass(NullWritable.class);
|
||||
groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class);
|
||||
groupByJob.setNumReduceTasks(1);
|
||||
JobHelper.setupClasspath(config, groupByJob);
|
||||
|
||||
config.addInputPaths(groupByJob);
|
||||
config.intoConfiguration(groupByJob);
|
||||
FileOutputFormat.setOutputPath(groupByJob, config.makeGroupedDataDir());
|
||||
|
||||
groupByJob.submit();
|
||||
log.info("Job %s submitted, status available at: %s", groupByJob.getJobName(), groupByJob.getTrackingURL());
|
||||
|
||||
if (!groupByJob.waitForCompletion(true)) {
|
||||
log.error("Job failed: %s", groupByJob.getJobID());
|
||||
return false;
|
||||
}
|
||||
|
||||
/*
|
||||
* Load partitions and intervals determined by the previous job.
|
||||
*/
|
||||
|
||||
log.info("Job completed, loading up partitions for intervals[%s].", config.getSegmentGranularIntervals());
|
||||
FileSystem fileSystem = null;
|
||||
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<Interval> intervals = config.jsonMapper.readValue(
|
||||
Utils.openInputStream(groupByJob, intervalInfoPath), new TypeReference<List<Interval>>()
|
||||
{
|
||||
}
|
||||
);
|
||||
config.setGranularitySpec(new UniformGranularitySpec(config.getGranularitySpec().getGranularity(), intervals));
|
||||
log.info("Determined Intervals for Job [%s]" + config.getSegmentGranularIntervals());
|
||||
}
|
||||
Map<DateTime, List<HadoopyShardSpec>> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance());
|
||||
int shardCount = 0;
|
||||
for (Interval segmentGranularity : config.getSegmentGranularIntervals().get()) {
|
||||
DateTime bucket = segmentGranularity.getStart();
|
||||
|
||||
final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(segmentGranularity);
|
||||
if (fileSystem == null) {
|
||||
fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration());
|
||||
}
|
||||
if (fileSystem.exists(partitionInfoPath)) {
|
||||
Long cardinality = config.jsonMapper.readValue(
|
||||
Utils.openInputStream(groupByJob, partitionInfoPath), new TypeReference<Long>()
|
||||
{
|
||||
}
|
||||
);
|
||||
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<HadoopyShardSpec> 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);
|
||||
}
|
||||
}
|
||||
config.setShardSpecs(shardSpecs);
|
||||
log.info(
|
||||
"DetermineHashedPartitionsJob took %d millis",
|
||||
(System.currentTimeMillis() - startTime)
|
||||
);
|
||||
|
||||
return true;
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static class DetermineCardinalityMapper extends HadoopDruidIndexerMapper<LongWritable, BytesWritable>
|
||||
{
|
||||
private static HashFunction hashFunction = Hashing.murmur3_128();
|
||||
private QueryGranularity rollupGranularity = null;
|
||||
private Map<Interval, HyperLogLog> hyperLogLogs;
|
||||
private HadoopDruidIndexerConfig config;
|
||||
private boolean determineIntervals;
|
||||
|
||||
@Override
|
||||
protected void setup(Context context)
|
||||
throws IOException, InterruptedException
|
||||
{
|
||||
super.setup(context);
|
||||
rollupGranularity = getConfig().getRollupSpec().getRollupGranularity();
|
||||
config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration());
|
||||
Optional<Set<Interval>> intervals = config.getSegmentGranularIntervals();
|
||||
if (intervals.isPresent()) {
|
||||
determineIntervals = false;
|
||||
final ImmutableMap.Builder<Interval, HyperLogLog> builder = ImmutableMap.builder();
|
||||
for (final Interval bucketInterval : intervals.get()) {
|
||||
builder.put(bucketInterval, new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE));
|
||||
}
|
||||
hyperLogLogs = builder.build();
|
||||
} else {
|
||||
determineIntervals = true;
|
||||
hyperLogLogs = Maps.newHashMap();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerMap(
|
||||
InputRow inputRow,
|
||||
Text text,
|
||||
Context context
|
||||
) throws IOException, InterruptedException
|
||||
{
|
||||
|
||||
final List<Object> groupKey = Rows.toGroupKey(
|
||||
rollupGranularity.truncate(inputRow.getTimestampFromEpoch()),
|
||||
inputRow
|
||||
);
|
||||
Interval interval;
|
||||
if (determineIntervals) {
|
||||
interval = config.getGranularitySpec().getGranularity().bucket(new DateTime(inputRow.getTimestampFromEpoch()));
|
||||
|
||||
if (!hyperLogLogs.containsKey(interval)) {
|
||||
hyperLogLogs.put(interval, new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE));
|
||||
}
|
||||
} else {
|
||||
final Optional<Interval> maybeInterval = config.getGranularitySpec()
|
||||
.bucketInterval(new DateTime(inputRow.getTimestampFromEpoch()));
|
||||
|
||||
if (!maybeInterval.isPresent()) {
|
||||
throw new ISE("WTF?! No bucket found for timestamp: %s", inputRow.getTimestampFromEpoch());
|
||||
}
|
||||
interval = maybeInterval.get();
|
||||
}
|
||||
hyperLogLogs.get(interval)
|
||||
.offerHashed(
|
||||
hashFunction.hashBytes(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey))
|
||||
.asLong()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(Context context) throws IOException, InterruptedException
|
||||
{
|
||||
setup(context);
|
||||
|
||||
while (context.nextKeyValue()) {
|
||||
map(context.getCurrentKey(), context.getCurrentValue(), context);
|
||||
}
|
||||
|
||||
for (Map.Entry<Interval, HyperLogLog> entry : hyperLogLogs.entrySet()) {
|
||||
context.write(
|
||||
new LongWritable(entry.getKey().getStartMillis()),
|
||||
new BytesWritable(entry.getValue().getBytes())
|
||||
);
|
||||
}
|
||||
cleanup(context);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static class DetermineCardinalityReducer
|
||||
extends Reducer<LongWritable, BytesWritable, NullWritable, NullWritable>
|
||||
{
|
||||
private final List<Interval> intervals = Lists.newArrayList();
|
||||
protected HadoopDruidIndexerConfig config = null;
|
||||
|
||||
@Override
|
||||
protected void setup(Context context)
|
||||
throws IOException, InterruptedException
|
||||
{
|
||||
config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void reduce(
|
||||
LongWritable key,
|
||||
Iterable<BytesWritable> values,
|
||||
Context context
|
||||
) throws IOException, InterruptedException
|
||||
{
|
||||
HyperLogLog aggregate = new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE);
|
||||
for (BytesWritable value : values) {
|
||||
HyperLogLog logValue = HyperLogLog.Builder.build(value.getBytes());
|
||||
try {
|
||||
aggregate.addAll(logValue);
|
||||
}
|
||||
catch (CardinalityMergeException e) {
|
||||
e.printStackTrace(); // TODO: check for better handling
|
||||
}
|
||||
}
|
||||
Interval interval = config.getGranularitySpec().getGranularity().bucket(new DateTime(key.get()));
|
||||
intervals.add(interval);
|
||||
final Path outPath = config.makeSegmentPartitionInfoPath(interval);
|
||||
final OutputStream out = Utils.makePathAndOutputStream(
|
||||
context, outPath, config.isOverwriteFiles()
|
||||
);
|
||||
|
||||
try {
|
||||
HadoopDruidIndexerConfig.jsonMapper.writerWithType(
|
||||
new TypeReference<Long>()
|
||||
{
|
||||
}
|
||||
).writeValue(
|
||||
out,
|
||||
aggregate.cardinality()
|
||||
);
|
||||
}
|
||||
finally {
|
||||
Closeables.close(out, false);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(Context context)
|
||||
throws IOException, InterruptedException
|
||||
{
|
||||
super.run(context);
|
||||
if (!config.getSegmentGranularIntervals().isPresent()) {
|
||||
final Path outPath = config.makeIntervalInfoPath();
|
||||
final OutputStream out = Utils.makePathAndOutputStream(
|
||||
context, outPath, config.isOverwriteFiles()
|
||||
);
|
||||
|
||||
try {
|
||||
HadoopDruidIndexerConfig.jsonMapper.writerWithType(
|
||||
new TypeReference<List<Interval>>()
|
||||
{
|
||||
}
|
||||
).writeValue(
|
||||
out,
|
||||
intervals
|
||||
);
|
||||
}
|
||||
finally {
|
||||
Closeables.close(out, false);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -26,9 +26,7 @@ import com.google.common.base.Optional;
|
|||
import com.google.common.base.Splitter;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ComparisonChain;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSortedSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -40,7 +38,9 @@ import com.metamx.common.guava.nary.BinaryFn;
|
|||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.collections.CombiningIterable;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.Rows;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import io.druid.timeline.partition.ShardSpec;
|
||||
import io.druid.timeline.partition.SingleDimensionShardSpec;
|
||||
|
@ -76,7 +76,6 @@ import java.util.Comparator;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Determines appropriate ShardSpecs for a job by determining whether or not partitioning is necessary, and if so,
|
||||
|
@ -107,16 +106,6 @@ public class DeterminePartitionsJob implements Jobby
|
|||
this.config = config;
|
||||
}
|
||||
|
||||
public static void injectSystemProperties(Job job)
|
||||
{
|
||||
final Configuration conf = job.getConfiguration();
|
||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
||||
if (propName.startsWith("hadoop.")) {
|
||||
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public boolean run()
|
||||
{
|
||||
try {
|
||||
|
@ -125,13 +114,17 @@ public class DeterminePartitionsJob implements Jobby
|
|||
* in the final segment.
|
||||
*/
|
||||
|
||||
if(!(config.getPartitionsSpec() instanceof SingleDimensionPartitionsSpec)){
|
||||
throw new ISE("DeterminePartitionsJob can only be run for SingleDimensionPartitionsSpec, partitionSpec found [%s]", config.getPartitionsSpec());
|
||||
}
|
||||
|
||||
if (!config.getPartitionsSpec().isAssumeGrouped()) {
|
||||
final Job groupByJob = new Job(
|
||||
new Configuration(),
|
||||
String.format("%s-determine_partitions_groupby-%s", config.getDataSource(), config.getIntervals())
|
||||
);
|
||||
|
||||
injectSystemProperties(groupByJob);
|
||||
JobHelper.injectSystemProperties(groupByJob);
|
||||
groupByJob.setInputFormatClass(TextInputFormat.class);
|
||||
groupByJob.setMapperClass(DeterminePartitionsGroupByMapper.class);
|
||||
groupByJob.setMapOutputKeyClass(BytesWritable.class);
|
||||
|
@ -168,7 +161,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
|
||||
dimSelectionJob.getConfiguration().set("io.sort.record.percent", "0.19");
|
||||
|
||||
injectSystemProperties(dimSelectionJob);
|
||||
JobHelper.injectSystemProperties(dimSelectionJob);
|
||||
|
||||
if (!config.getPartitionsSpec().isAssumeGrouped()) {
|
||||
// Read grouped data from the groupByJob.
|
||||
|
@ -190,7 +183,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
dimSelectionJob.setOutputValueClass(Text.class);
|
||||
dimSelectionJob.setOutputFormatClass(DeterminePartitionsDimSelectionOutputFormat.class);
|
||||
dimSelectionJob.setPartitionerClass(DeterminePartitionsDimSelectionPartitioner.class);
|
||||
dimSelectionJob.setNumReduceTasks(config.getGranularitySpec().bucketIntervals().size());
|
||||
dimSelectionJob.setNumReduceTasks(config.getGranularitySpec().bucketIntervals().get().size());
|
||||
JobHelper.setupClasspath(config, dimSelectionJob);
|
||||
|
||||
config.intoConfiguration(dimSelectionJob);
|
||||
|
@ -216,10 +209,8 @@ public class DeterminePartitionsJob implements Jobby
|
|||
FileSystem fileSystem = null;
|
||||
Map<DateTime, List<HadoopyShardSpec>> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance());
|
||||
int shardCount = 0;
|
||||
for (Interval segmentGranularity : config.getSegmentGranularIntervals()) {
|
||||
DateTime bucket = segmentGranularity.getStart();
|
||||
|
||||
final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0));
|
||||
for (Interval segmentGranularity : config.getSegmentGranularIntervals().get()) {
|
||||
final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(segmentGranularity);
|
||||
if (fileSystem == null) {
|
||||
fileSystem = partitionInfoPath.getFileSystem(dimSelectionJob.getConfiguration());
|
||||
}
|
||||
|
@ -233,10 +224,10 @@ public class DeterminePartitionsJob implements Jobby
|
|||
List<HadoopyShardSpec> 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]", bucket, i, actualSpecs.get(i));
|
||||
log.info("DateTime[%s], partition[%d], spec[%s]", segmentGranularity, i, actualSpecs.get(i));
|
||||
}
|
||||
|
||||
shardSpecs.put(bucket, actualSpecs);
|
||||
shardSpecs.put(segmentGranularity.getStart(), actualSpecs);
|
||||
} else {
|
||||
log.info("Path[%s] didn't exist!?", partitionInfoPath);
|
||||
}
|
||||
|
@ -269,17 +260,9 @@ public class DeterminePartitionsJob implements Jobby
|
|||
Context context
|
||||
) throws IOException, InterruptedException
|
||||
{
|
||||
// Create group key, there are probably more efficient ways of doing this
|
||||
final Map<String, Set<String>> dims = Maps.newTreeMap();
|
||||
for (final String dim : inputRow.getDimensions()) {
|
||||
final Set<String> dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim));
|
||||
if (dimValues.size() > 0) {
|
||||
dims.put(dim, dimValues);
|
||||
}
|
||||
}
|
||||
final List<Object> groupKey = ImmutableList.of(
|
||||
final List<Object> groupKey = Rows.toGroupKey(
|
||||
rollupGranularity.truncate(inputRow.getTimestampFromEpoch()),
|
||||
dims
|
||||
inputRow
|
||||
);
|
||||
context.write(
|
||||
new BytesWritable(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)),
|
||||
|
@ -315,8 +298,8 @@ public class DeterminePartitionsJob implements Jobby
|
|||
throws IOException, InterruptedException
|
||||
{
|
||||
final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration());
|
||||
final String partitionDimension = config.getPartitionDimension();
|
||||
helper = new DeterminePartitionsDimSelectionMapperHelper(config, partitionDimension);
|
||||
SingleDimensionPartitionsSpec spec = (SingleDimensionPartitionsSpec) config.getPartitionsSpec();
|
||||
helper = new DeterminePartitionsDimSelectionMapperHelper(config, spec.getPartitionDimension());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -347,8 +330,8 @@ public class DeterminePartitionsJob implements Jobby
|
|||
{
|
||||
super.setup(context);
|
||||
final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration());
|
||||
final String partitionDimension = config.getPartitionDimension();
|
||||
helper = new DeterminePartitionsDimSelectionMapperHelper(config, partitionDimension);
|
||||
final SingleDimensionPartitionsSpec spec = (SingleDimensionPartitionsSpec) config.getPartitionsSpec();
|
||||
helper = new DeterminePartitionsDimSelectionMapperHelper(config, spec.getPartitionDimension());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -383,7 +366,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
|
||||
final ImmutableMap.Builder<DateTime, Integer> timeIndexBuilder = ImmutableMap.builder();
|
||||
int idx = 0;
|
||||
for (final Interval bucketInterval : config.getGranularitySpec().bucketIntervals()) {
|
||||
for (final Interval bucketInterval : config.getGranularitySpec().bucketIntervals().get()) {
|
||||
timeIndexBuilder.put(bucketInterval.getStart(), idx);
|
||||
idx++;
|
||||
}
|
||||
|
@ -741,7 +724,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
}
|
||||
|
||||
final OutputStream out = Utils.makePathAndOutputStream(
|
||||
context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles()
|
||||
context, config.makeSegmentPartitionInfoPath(config.getGranularitySpec().bucketInterval(bucket).get()), config.isOverwriteFiles()
|
||||
);
|
||||
|
||||
final DimPartitions chosenPartitions = maxCardinality > HIGH_CARDINALITY_THRESHOLD
|
||||
|
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* 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.indexer;
|
||||
|
||||
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.NoneShardSpec;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeComparator;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class HadoopDruidDetermineConfigurationJob implements Jobby
|
||||
{
|
||||
private static final Logger log = new Logger(HadoopDruidDetermineConfigurationJob.class);
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
|
||||
@Inject
|
||||
public HadoopDruidDetermineConfigurationJob(
|
||||
HadoopDruidIndexerConfig config
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean run()
|
||||
{
|
||||
List<Jobby> jobs = Lists.newArrayList();
|
||||
|
||||
JobHelper.ensurePaths(config);
|
||||
|
||||
if (config.isDeterminingPartitions()) {
|
||||
jobs.add(config.getPartitionsSpec().getPartitionJob(config));
|
||||
} else {
|
||||
Map<DateTime, List<HadoopyShardSpec>> 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);
|
||||
}
|
||||
config.setShardSpecs(shardSpecs);
|
||||
}
|
||||
|
||||
return JobHelper.runJobs(jobs, config);
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -50,6 +50,7 @@ import io.druid.guice.annotations.Self;
|
|||
import io.druid.indexer.granularity.GranularitySpec;
|
||||
import io.druid.indexer.granularity.UniformGranularitySpec;
|
||||
import io.druid.indexer.partitions.PartitionsSpec;
|
||||
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||
import io.druid.indexer.path.PathSpec;
|
||||
import io.druid.indexer.rollup.DataRollupSpec;
|
||||
import io.druid.indexer.updater.DbUpdaterJobSpec;
|
||||
|
@ -73,6 +74,7 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -178,7 +180,7 @@ public class HadoopDruidIndexerConfig
|
|||
this.partitionsSpec = partitionsSpec;
|
||||
} else {
|
||||
// Backwards compatibility
|
||||
this.partitionsSpec = new PartitionsSpec(partitionDimension, targetPartitionSize, null, false);
|
||||
this.partitionsSpec = new SingleDimensionPartitionsSpec(partitionDimension, targetPartitionSize, null, false);
|
||||
}
|
||||
|
||||
if (granularitySpec != null) {
|
||||
|
@ -378,17 +380,17 @@ public class HadoopDruidIndexerConfig
|
|||
this.ignoreInvalidRows = ignoreInvalidRows;
|
||||
}
|
||||
|
||||
public List<Interval> getIntervals()
|
||||
public Optional<List<Interval>> getIntervals()
|
||||
{
|
||||
return JodaUtils.condenseIntervals(getGranularitySpec().bucketIntervals());
|
||||
Optional<SortedSet<Interval>> setOptional = getGranularitySpec().bucketIntervals();
|
||||
if (setOptional.isPresent()) {
|
||||
return Optional.of((List<Interval>) JodaUtils.condenseIntervals(setOptional.get()));
|
||||
} else {
|
||||
return Optional.absent();
|
||||
}
|
||||
}
|
||||
|
||||
public String getPartitionDimension()
|
||||
{
|
||||
return partitionsSpec.getPartitionDimension();
|
||||
}
|
||||
|
||||
public boolean partitionByDimension()
|
||||
public boolean isDeterminingPartitions()
|
||||
{
|
||||
return partitionsSpec.isDeterminingPartitions();
|
||||
}
|
||||
|
@ -431,7 +433,7 @@ public class HadoopDruidIndexerConfig
|
|||
);
|
||||
}
|
||||
|
||||
return new StringInputRowParser(getTimestampSpec(), getDataSpec(), dimensionExclusions);
|
||||
return new StringInputRowParser(null, getTimestampSpec(), getDataSpec(), dimensionExclusions, null);
|
||||
}
|
||||
|
||||
public HadoopyShardSpec getShardSpec(Bucket bucket)
|
||||
|
@ -483,64 +485,70 @@ public class HadoopDruidIndexerConfig
|
|||
throw new ISE("row[%s] doesn't fit in any shard[%s]", inputRow, shards);
|
||||
}
|
||||
|
||||
public Set<Interval> getSegmentGranularIntervals()
|
||||
public Optional<Set<Interval>> getSegmentGranularIntervals()
|
||||
{
|
||||
return granularitySpec.bucketIntervals();
|
||||
return Optional.fromNullable((Set<Interval>) granularitySpec.bucketIntervals().orNull());
|
||||
}
|
||||
|
||||
public Iterable<Bucket> getAllBuckets()
|
||||
public Optional<Iterable<Bucket>> getAllBuckets()
|
||||
{
|
||||
return FunctionalIterable
|
||||
.create(getSegmentGranularIntervals())
|
||||
.transformCat(
|
||||
new Function<Interval, Iterable<Bucket>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<Bucket> apply(Interval input)
|
||||
{
|
||||
final DateTime bucketTime = input.getStart();
|
||||
final List<HadoopyShardSpec> specs = shardSpecs.get(bucketTime);
|
||||
if (specs == null) {
|
||||
return ImmutableList.of();
|
||||
}
|
||||
Optional<Set<Interval>> intervals = getSegmentGranularIntervals();
|
||||
if (intervals.isPresent()) {
|
||||
return Optional.of(
|
||||
(Iterable<Bucket>) FunctionalIterable
|
||||
.create(intervals.get())
|
||||
.transformCat(
|
||||
new Function<Interval, Iterable<Bucket>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<Bucket> apply(Interval input)
|
||||
{
|
||||
final DateTime bucketTime = input.getStart();
|
||||
final List<HadoopyShardSpec> specs = shardSpecs.get(bucketTime);
|
||||
if (specs == null) {
|
||||
return ImmutableList.of();
|
||||
}
|
||||
|
||||
return FunctionalIterable
|
||||
.create(specs)
|
||||
.transform(
|
||||
new Function<HadoopyShardSpec, Bucket>()
|
||||
{
|
||||
int i = 0;
|
||||
return FunctionalIterable
|
||||
.create(specs)
|
||||
.transform(
|
||||
new Function<HadoopyShardSpec, Bucket>()
|
||||
{
|
||||
int i = 0;
|
||||
|
||||
@Override
|
||||
public Bucket apply(HadoopyShardSpec input)
|
||||
{
|
||||
return new Bucket(input.getShardNum(), bucketTime, i++);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
@Override
|
||||
public Bucket apply(HadoopyShardSpec input)
|
||||
{
|
||||
return new Bucket(input.getShardNum(), bucketTime, i++);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
} else {
|
||||
return Optional.absent();
|
||||
}
|
||||
}
|
||||
|
||||
/******************************************
|
||||
Path helper logic
|
||||
******************************************/
|
||||
/******************************************
|
||||
Path helper logic
|
||||
******************************************/
|
||||
|
||||
/**
|
||||
* Make the intermediate path for this job run.
|
||||
*
|
||||
* @return the intermediate path for this job run.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Make the intermediate path for this job run.
|
||||
*
|
||||
* @return the intermediate path for this job run.
|
||||
*/
|
||||
public Path makeIntermediatePath()
|
||||
{
|
||||
return new Path(String.format("%s/%s/%s", getWorkingPath(), getDataSource(), getVersion().replace(":", "")));
|
||||
}
|
||||
|
||||
public Path makeSegmentPartitionInfoPath(Bucket bucket)
|
||||
public Path makeSegmentPartitionInfoPath(Interval bucketInterval)
|
||||
{
|
||||
final Interval bucketInterval = getGranularitySpec().bucketInterval(bucket.time).get();
|
||||
|
||||
return new Path(
|
||||
String.format(
|
||||
"%s/%s_%s/partitions.json",
|
||||
|
@ -551,6 +559,16 @@ public class HadoopDruidIndexerConfig
|
|||
);
|
||||
}
|
||||
|
||||
public Path makeIntervalInfoPath()
|
||||
{
|
||||
return new Path(
|
||||
String.format(
|
||||
"%s/intervals.json",
|
||||
makeIntermediatePath()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public Path makeDescriptorInfoDir()
|
||||
{
|
||||
return new Path(makeIntermediatePath(), "segmentDescriptorInfo");
|
||||
|
@ -625,8 +643,5 @@ public class HadoopDruidIndexerConfig
|
|||
Preconditions.checkNotNull(segmentOutputPath, "segmentOutputPath");
|
||||
Preconditions.checkNotNull(version, "version");
|
||||
Preconditions.checkNotNull(rollupSpec, "rollupSpec");
|
||||
|
||||
final int nIntervals = getIntervals().size();
|
||||
Preconditions.checkArgument(nIntervals > 0, "intervals.size()[%s] <= 0", nIntervals);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,6 +46,10 @@ public class HadoopDruidIndexerConfigBuilder
|
|||
return HadoopDruidIndexerConfig.jsonMapper.convertValue(schema, HadoopDruidIndexerConfig.class);
|
||||
}
|
||||
|
||||
public static HadoopDruidIndexerSchema toSchema(HadoopDruidIndexerConfig config){
|
||||
return HadoopDruidIndexerConfig.jsonMapper.convertValue(config, HadoopDruidIndexerSchema.class);
|
||||
}
|
||||
|
||||
public static HadoopDruidIndexerConfig fromMap(Map<String, Object> argSpec)
|
||||
{
|
||||
return HadoopDruidIndexerConfig.jsonMapper.convertValue(argSpec, HadoopDruidIndexerConfig.class);
|
||||
|
|
|
@ -19,34 +19,20 @@
|
|||
|
||||
package io.druid.indexer;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeComparator;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class HadoopDruidIndexerJob implements Jobby
|
||||
{
|
||||
private static final Logger log = new Logger(HadoopDruidIndexerJob.class);
|
||||
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
private final DbUpdaterJob dbUpdaterJob;
|
||||
|
||||
private IndexGeneratorJob indexJob;
|
||||
private volatile List<DataSegment> publishedSegments = null;
|
||||
|
||||
|
@ -69,23 +55,7 @@ public class HadoopDruidIndexerJob implements Jobby
|
|||
public boolean run()
|
||||
{
|
||||
List<Jobby> jobs = Lists.newArrayList();
|
||||
|
||||
ensurePaths();
|
||||
|
||||
if (config.partitionByDimension()) {
|
||||
jobs.add(new DeterminePartitionsJob(config));
|
||||
}
|
||||
else {
|
||||
Map<DateTime, List<HadoopyShardSpec>> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance());
|
||||
int shardCount = 0;
|
||||
for (Interval segmentGranularity : config.getSegmentGranularIntervals()) {
|
||||
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);
|
||||
}
|
||||
config.setShardSpecs(shardSpecs);
|
||||
}
|
||||
JobHelper.ensurePaths(config);
|
||||
|
||||
indexJob = new IndexGeneratorJob(config);
|
||||
jobs.add(indexJob);
|
||||
|
@ -96,65 +66,24 @@ public class HadoopDruidIndexerJob implements Jobby
|
|||
log.info("No updaterJobSpec set, not uploading to database");
|
||||
}
|
||||
|
||||
String failedMessage = null;
|
||||
for (Jobby job : jobs) {
|
||||
if (failedMessage == null) {
|
||||
if (!job.run()) {
|
||||
failedMessage = String.format("Job[%s] failed!", job.getClass());
|
||||
}
|
||||
jobs.add(new Jobby()
|
||||
{
|
||||
@Override
|
||||
public boolean run()
|
||||
{
|
||||
publishedSegments = IndexGeneratorJob.getPublishedSegments(config);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
if (failedMessage == null) {
|
||||
publishedSegments = IndexGeneratorJob.getPublishedSegments(config);
|
||||
}
|
||||
|
||||
if (!config.isLeaveIntermediate()) {
|
||||
if (failedMessage == null || config.isCleanupOnFailure()) {
|
||||
Path workingPath = config.makeIntermediatePath();
|
||||
log.info("Deleting path[%s]", workingPath);
|
||||
try {
|
||||
workingPath.getFileSystem(new Configuration()).delete(workingPath, true);
|
||||
}
|
||||
catch (IOException e) {
|
||||
log.error(e, "Failed to cleanup path[%s]", workingPath);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (failedMessage != null) {
|
||||
throw new ISE(failedMessage);
|
||||
}
|
||||
|
||||
JobHelper.runJobs(jobs, config);
|
||||
return true;
|
||||
}
|
||||
|
||||
private void ensurePaths()
|
||||
public List<DataSegment> getPublishedSegments()
|
||||
{
|
||||
// config.addInputPaths() can have side-effects ( boo! :( ), so this stuff needs to be done before anything else
|
||||
try {
|
||||
Job job = new Job(
|
||||
new Configuration(),
|
||||
String.format("%s-determine_partitions-%s", config.getDataSource(), config.getIntervals())
|
||||
);
|
||||
|
||||
job.getConfiguration().set("io.sort.record.percent", "0.19");
|
||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
||||
Configuration conf = job.getConfiguration();
|
||||
if (propName.startsWith("hadoop.")) {
|
||||
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
||||
}
|
||||
}
|
||||
|
||||
config.addInputPaths(job);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public List<DataSegment> getPublishedSegments() {
|
||||
if(publishedSegments == null) {
|
||||
if (publishedSegments == null) {
|
||||
throw new IllegalStateException("Job hasn't run yet. No segments have been published yet.");
|
||||
}
|
||||
return publishedSegments;
|
||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.indexer;
|
|||
import com.metamx.common.RE;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.StringInputRowParser;
|
||||
import io.druid.indexer.granularity.GranularitySpec;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapreduce.Mapper;
|
||||
|
@ -70,8 +71,9 @@ public abstract class HadoopDruidIndexerMapper<KEYOUT, VALUEOUT> extends Mapper<
|
|||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
if(config.getGranularitySpec().bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())).isPresent()) {
|
||||
GranularitySpec spec = config.getGranularitySpec();
|
||||
if (!spec.bucketIntervals().isPresent() || spec.bucketInterval(new DateTime(inputRow.getTimestampFromEpoch()))
|
||||
.isPresent()) {
|
||||
innerMap(inputRow, value, context);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import io.druid.segment.incremental.IncrementalIndex;
|
|||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
@ -84,9 +85,7 @@ import java.util.zip.ZipOutputStream;
|
|||
public class IndexGeneratorJob implements Jobby
|
||||
{
|
||||
private static final Logger log = new Logger(IndexGeneratorJob.class);
|
||||
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
|
||||
private IndexGeneratorStats jobStats;
|
||||
|
||||
public IndexGeneratorJob(
|
||||
|
@ -97,65 +96,6 @@ public class IndexGeneratorJob implements Jobby
|
|||
this.jobStats = new IndexGeneratorStats();
|
||||
}
|
||||
|
||||
public IndexGeneratorStats getJobStats()
|
||||
{
|
||||
return jobStats;
|
||||
}
|
||||
|
||||
public boolean run()
|
||||
{
|
||||
try {
|
||||
Job job = new Job(
|
||||
new Configuration(),
|
||||
String.format("%s-index-generator-%s", config.getDataSource(), config.getIntervals())
|
||||
);
|
||||
|
||||
job.getConfiguration().set("io.sort.record.percent", "0.23");
|
||||
|
||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
||||
Configuration conf = job.getConfiguration();
|
||||
if (propName.startsWith("hadoop.")) {
|
||||
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
||||
}
|
||||
}
|
||||
|
||||
job.setInputFormatClass(TextInputFormat.class);
|
||||
|
||||
job.setMapperClass(IndexGeneratorMapper.class);
|
||||
job.setMapOutputValueClass(Text.class);
|
||||
|
||||
SortableBytes.useSortableBytesAsMapOutputKey(job);
|
||||
|
||||
job.setNumReduceTasks(Iterables.size(config.getAllBuckets()));
|
||||
job.setPartitionerClass(IndexGeneratorPartitioner.class);
|
||||
|
||||
job.setReducerClass(IndexGeneratorReducer.class);
|
||||
job.setOutputKeyClass(BytesWritable.class);
|
||||
job.setOutputValueClass(Text.class);
|
||||
job.setOutputFormatClass(IndexGeneratorOutputFormat.class);
|
||||
FileOutputFormat.setOutputPath(job, config.makeIntermediatePath());
|
||||
|
||||
config.addInputPaths(job);
|
||||
config.intoConfiguration(job);
|
||||
|
||||
JobHelper.setupClasspath(config, job);
|
||||
|
||||
job.submit();
|
||||
log.info("Job %s submitted, status available at %s", job.getJobName(), job.getTrackingURL());
|
||||
|
||||
boolean success = job.waitForCompletion(true);
|
||||
|
||||
Counter invalidRowCount = job.getCounters()
|
||||
.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER);
|
||||
jobStats.setInvalidRowCount(invalidRowCount.getValue());
|
||||
|
||||
return success;
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static List<DataSegment> getPublishedSegments(HadoopDruidIndexerConfig config)
|
||||
{
|
||||
|
||||
|
@ -189,6 +129,60 @@ public class IndexGeneratorJob implements Jobby
|
|||
return publishedSegments;
|
||||
}
|
||||
|
||||
public IndexGeneratorStats getJobStats()
|
||||
{
|
||||
return jobStats;
|
||||
}
|
||||
|
||||
public boolean run()
|
||||
{
|
||||
try {
|
||||
Job job = new Job(
|
||||
new Configuration(),
|
||||
String.format("%s-index-generator-%s", config.getDataSource(), config.getIntervals())
|
||||
);
|
||||
|
||||
job.getConfiguration().set("io.sort.record.percent", "0.23");
|
||||
|
||||
JobHelper.injectSystemProperties(job);
|
||||
|
||||
job.setInputFormatClass(TextInputFormat.class);
|
||||
|
||||
job.setMapperClass(IndexGeneratorMapper.class);
|
||||
job.setMapOutputValueClass(Text.class);
|
||||
|
||||
SortableBytes.useSortableBytesAsMapOutputKey(job);
|
||||
|
||||
job.setNumReduceTasks(Iterables.size(config.getAllBuckets().get()));
|
||||
job.setPartitionerClass(IndexGeneratorPartitioner.class);
|
||||
|
||||
job.setReducerClass(IndexGeneratorReducer.class);
|
||||
job.setOutputKeyClass(BytesWritable.class);
|
||||
job.setOutputValueClass(Text.class);
|
||||
job.setOutputFormatClass(IndexGeneratorOutputFormat.class);
|
||||
FileOutputFormat.setOutputPath(job, config.makeIntermediatePath());
|
||||
|
||||
config.addInputPaths(job);
|
||||
config.intoConfiguration(job);
|
||||
|
||||
JobHelper.setupClasspath(config, job);
|
||||
|
||||
job.submit();
|
||||
log.info("Job %s submitted, status available at %s", job.getJobName(), job.getTrackingURL());
|
||||
|
||||
boolean success = job.waitForCompletion(true);
|
||||
|
||||
Counter invalidRowCount = job.getCounters()
|
||||
.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER);
|
||||
jobStats.setInvalidRowCount(invalidRowCount.getValue());
|
||||
|
||||
return success;
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static class IndexGeneratorMapper extends HadoopDruidIndexerMapper<BytesWritable, Text>
|
||||
|
||||
{
|
||||
|
@ -216,8 +210,9 @@ public class IndexGeneratorJob implements Jobby
|
|||
}
|
||||
}
|
||||
|
||||
public static class IndexGeneratorPartitioner extends Partitioner<BytesWritable, Text>
|
||||
public static class IndexGeneratorPartitioner extends Partitioner<BytesWritable, Text> implements Configurable
|
||||
{
|
||||
private Configuration config;
|
||||
|
||||
@Override
|
||||
public int getPartition(BytesWritable bytesWritable, Text text, int numPartitions)
|
||||
|
@ -225,12 +220,27 @@ public class IndexGeneratorJob implements Jobby
|
|||
final ByteBuffer bytes = ByteBuffer.wrap(bytesWritable.getBytes());
|
||||
bytes.position(4); // Skip length added by SortableBytes
|
||||
int shardNum = bytes.getInt();
|
||||
if (config.get("mapred.job.tracker").equals("local")) {
|
||||
return shardNum % numPartitions;
|
||||
} else {
|
||||
if (shardNum >= numPartitions) {
|
||||
throw new ISE("Not enough partitions, shard[%,d] >= numPartitions[%,d]", shardNum, numPartitions);
|
||||
}
|
||||
return shardNum;
|
||||
|
||||
if (shardNum >= numPartitions) {
|
||||
throw new ISE("Not enough partitions, shard[%,d] >= numPartitions[%,d]", shardNum, numPartitions);
|
||||
}
|
||||
}
|
||||
|
||||
return shardNum;
|
||||
@Override
|
||||
public Configuration getConf()
|
||||
{
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration config)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -20,9 +20,11 @@
|
|||
package io.druid.indexer;
|
||||
|
||||
import com.google.api.client.util.Sets;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.common.io.OutputSupplier;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.filecache.DistributedCache;
|
||||
|
@ -34,6 +36,7 @@ import org.apache.hadoop.mapreduce.Job;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -94,4 +97,63 @@ public class JobHelper
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void injectSystemProperties(Job job)
|
||||
{
|
||||
final Configuration conf = job.getConfiguration();
|
||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
||||
if (propName.startsWith("hadoop.")) {
|
||||
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void ensurePaths(HadoopDruidIndexerConfig config)
|
||||
{
|
||||
// config.addInputPaths() can have side-effects ( boo! :( ), so this stuff needs to be done before anything else
|
||||
try {
|
||||
Job job = new Job(
|
||||
new Configuration(),
|
||||
String.format("%s-determine_partitions-%s", config.getDataSource(), config.getIntervals())
|
||||
);
|
||||
|
||||
job.getConfiguration().set("io.sort.record.percent", "0.19");
|
||||
injectSystemProperties(job);
|
||||
|
||||
config.addInputPaths(job);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean runJobs(List<Jobby> jobs, HadoopDruidIndexerConfig config){
|
||||
String failedMessage = null;
|
||||
for (Jobby job : jobs) {
|
||||
if (failedMessage == null) {
|
||||
if (!job.run()) {
|
||||
failedMessage = String.format("Job[%s] failed!", job.getClass());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!config.isLeaveIntermediate()) {
|
||||
if (failedMessage == null || config.isCleanupOnFailure()) {
|
||||
Path workingPath = config.makeIntermediatePath();
|
||||
log.info("Deleting path[%s]", workingPath);
|
||||
try {
|
||||
workingPath.getFileSystem(new Configuration()).delete(workingPath, true);
|
||||
}
|
||||
catch (IOException e) {
|
||||
log.error(e, "Failed to cleanup path[%s]", workingPath);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (failedMessage != null) {
|
||||
throw new ISE(failedMessage);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -72,9 +72,9 @@ public class ArbitraryGranularitySpec implements GranularitySpec
|
|||
|
||||
@Override
|
||||
@JsonProperty("intervals")
|
||||
public SortedSet<Interval> bucketIntervals()
|
||||
public Optional<SortedSet<Interval>> bucketIntervals()
|
||||
{
|
||||
return intervals;
|
||||
return Optional.of((SortedSet<Interval>) intervals);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -40,7 +40,7 @@ import java.util.SortedSet;
|
|||
public interface GranularitySpec
|
||||
{
|
||||
/** Set of all time groups, broken up on segment boundaries. Should be sorted by interval start and non-overlapping.*/
|
||||
public SortedSet<Interval> bucketIntervals();
|
||||
public Optional<SortedSet<Interval>> bucketIntervals();
|
||||
|
||||
/** Time-grouping interval corresponding to some instant, if any. */
|
||||
public Optional<Interval> bucketInterval(DateTime dt);
|
||||
|
|
|
@ -35,7 +35,7 @@ import java.util.SortedSet;
|
|||
public class UniformGranularitySpec implements GranularitySpec
|
||||
{
|
||||
final private Granularity granularity;
|
||||
final private List<Interval> inputIntervals;
|
||||
final private Iterable<Interval> inputIntervals;
|
||||
final private ArbitraryGranularitySpec wrappedSpec;
|
||||
|
||||
@JsonCreator
|
||||
|
@ -44,21 +44,28 @@ public class UniformGranularitySpec implements GranularitySpec
|
|||
@JsonProperty("intervals") List<Interval> inputIntervals
|
||||
)
|
||||
{
|
||||
List<Interval> granularIntervals = Lists.newArrayList();
|
||||
|
||||
for (Interval inputInterval : inputIntervals) {
|
||||
Iterables.addAll(granularIntervals, granularity.getIterable(inputInterval));
|
||||
}
|
||||
|
||||
this.granularity = granularity;
|
||||
this.inputIntervals = ImmutableList.copyOf(inputIntervals);
|
||||
this.wrappedSpec = new ArbitraryGranularitySpec(granularIntervals);
|
||||
if (inputIntervals != null) {
|
||||
List<Interval> granularIntervals = Lists.newArrayList();
|
||||
for (Interval inputInterval : inputIntervals) {
|
||||
Iterables.addAll(granularIntervals, granularity.getIterable(inputInterval));
|
||||
}
|
||||
this.inputIntervals = ImmutableList.copyOf(inputIntervals);
|
||||
this.wrappedSpec = new ArbitraryGranularitySpec(granularIntervals);
|
||||
} else {
|
||||
this.inputIntervals = null;
|
||||
this.wrappedSpec = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSet<Interval> bucketIntervals()
|
||||
public Optional<SortedSet<Interval>> bucketIntervals()
|
||||
{
|
||||
return wrappedSpec.bucketIntervals();
|
||||
if (wrappedSpec == null) {
|
||||
return Optional.absent();
|
||||
} else {
|
||||
return wrappedSpec.bucketIntervals();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -75,8 +82,8 @@ public class UniformGranularitySpec implements GranularitySpec
|
|||
}
|
||||
|
||||
@JsonProperty("intervals")
|
||||
public Iterable<Interval> getIntervals()
|
||||
public Optional<Iterable<Interval>> getIntervals()
|
||||
{
|
||||
return inputIntervals;
|
||||
return Optional.fromNullable(inputIntervals);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexer.partitions;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
|
||||
public abstract class AbstractPartitionsSpec implements PartitionsSpec
|
||||
{
|
||||
private static final double DEFAULT_OVERSIZE_THRESHOLD = 1.5;
|
||||
private final long targetPartitionSize;
|
||||
private final long maxPartitionSize;
|
||||
private final boolean assumeGrouped;
|
||||
|
||||
public AbstractPartitionsSpec(
|
||||
Long targetPartitionSize,
|
||||
Long maxPartitionSize,
|
||||
Boolean assumeGrouped
|
||||
)
|
||||
{
|
||||
this.targetPartitionSize = targetPartitionSize == null ? -1 : targetPartitionSize;
|
||||
this.maxPartitionSize = maxPartitionSize == null
|
||||
? (long) (this.targetPartitionSize * DEFAULT_OVERSIZE_THRESHOLD)
|
||||
: maxPartitionSize;
|
||||
this.assumeGrouped = assumeGrouped == null ? false : assumeGrouped;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getTargetPartitionSize()
|
||||
{
|
||||
return targetPartitionSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getMaxPartitionSize()
|
||||
{
|
||||
return maxPartitionSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isAssumeGrouped()
|
||||
{
|
||||
return assumeGrouped;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDeterminingPartitions()
|
||||
{
|
||||
return targetPartitionSize > 0;
|
||||
}
|
||||
}
|
|
@ -19,69 +19,33 @@
|
|||
|
||||
package io.druid.indexer.partitions;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfig;
|
||||
import io.druid.indexer.Jobby;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
public class PartitionsSpec
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = SingleDimensionPartitionsSpec.class)
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "dimension", value = SingleDimensionPartitionsSpec.class),
|
||||
@JsonSubTypes.Type(name = "random", value = RandomPartitionsSpec.class)
|
||||
})
|
||||
public interface PartitionsSpec
|
||||
{
|
||||
private static final double DEFAULT_OVERSIZE_THRESHOLD = 1.5;
|
||||
@JsonIgnore
|
||||
public Jobby getPartitionJob(HadoopDruidIndexerConfig config);
|
||||
|
||||
@Nullable
|
||||
private final String partitionDimension;
|
||||
@JsonProperty
|
||||
public long getTargetPartitionSize();
|
||||
|
||||
private final long targetPartitionSize;
|
||||
@JsonProperty
|
||||
public long getMaxPartitionSize();
|
||||
|
||||
private final long maxPartitionSize;
|
||||
|
||||
private final boolean assumeGrouped;
|
||||
|
||||
@JsonCreator
|
||||
public PartitionsSpec(
|
||||
@JsonProperty("partitionDimension") @Nullable String partitionDimension,
|
||||
@JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize,
|
||||
@JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize,
|
||||
@JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped
|
||||
)
|
||||
{
|
||||
this.partitionDimension = partitionDimension;
|
||||
this.targetPartitionSize = targetPartitionSize == null ? -1 : targetPartitionSize;
|
||||
this.maxPartitionSize = maxPartitionSize == null
|
||||
? (long) (this.targetPartitionSize * DEFAULT_OVERSIZE_THRESHOLD)
|
||||
: maxPartitionSize;
|
||||
this.assumeGrouped = assumeGrouped == null ? false : assumeGrouped;
|
||||
}
|
||||
@JsonProperty
|
||||
public boolean isAssumeGrouped();
|
||||
|
||||
@JsonIgnore
|
||||
public boolean isDeterminingPartitions()
|
||||
{
|
||||
return targetPartitionSize > 0;
|
||||
}
|
||||
public boolean isDeterminingPartitions();
|
||||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public String getPartitionDimension()
|
||||
{
|
||||
return partitionDimension;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getTargetPartitionSize()
|
||||
{
|
||||
return targetPartitionSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getMaxPartitionSize()
|
||||
{
|
||||
return maxPartitionSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isAssumeGrouped()
|
||||
{
|
||||
return assumeGrouped;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* 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.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;
|
||||
|
||||
public class RandomPartitionsSpec extends AbstractPartitionsSpec
|
||||
{
|
||||
@JsonCreator
|
||||
public RandomPartitionsSpec(
|
||||
@JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize,
|
||||
@JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize,
|
||||
@JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped
|
||||
)
|
||||
{
|
||||
super(targetPartitionSize, maxPartitionSize, assumeGrouped);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Jobby getPartitionJob(HadoopDruidIndexerConfig config)
|
||||
{
|
||||
return new DetermineHashedPartitionsJob(config);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexer.partitions;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.indexer.DeterminePartitionsJob;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfig;
|
||||
import io.druid.indexer.Jobby;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
public class SingleDimensionPartitionsSpec extends AbstractPartitionsSpec
|
||||
{
|
||||
@Nullable
|
||||
private final String partitionDimension;
|
||||
|
||||
@JsonCreator
|
||||
public SingleDimensionPartitionsSpec(
|
||||
@JsonProperty("partitionDimension") @Nullable String partitionDimension,
|
||||
@JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize,
|
||||
@JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize,
|
||||
@JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped
|
||||
)
|
||||
{
|
||||
super(targetPartitionSize, maxPartitionSize, assumeGrouped);
|
||||
this.partitionDimension = partitionDimension;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public String getPartitionDimension()
|
||||
{
|
||||
return partitionDimension;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Jobby getPartitionJob(HadoopDruidIndexerConfig config)
|
||||
{
|
||||
return new DeterminePartitionsJob(config);
|
||||
}
|
||||
}
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.indexer.path;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.Granularity;
|
||||
import com.metamx.common.guava.Comparators;
|
||||
|
@ -99,9 +100,12 @@ public class GranularityPathSpec implements PathSpec
|
|||
public Job addInputPaths(HadoopDruidIndexerConfig config, Job job) throws IOException
|
||||
{
|
||||
final Set<Interval> intervals = Sets.newTreeSet(Comparators.intervals());
|
||||
for (Interval segmentInterval : config.getSegmentGranularIntervals()) {
|
||||
for (Interval dataInterval : dataGranularity.getIterable(segmentInterval)) {
|
||||
intervals.add(dataInterval);
|
||||
Optional<Set<Interval>> optionalIntervals = config.getSegmentGranularIntervals();
|
||||
if (optionalIntervals.isPresent()) {
|
||||
for (Interval segmentInterval : optionalIntervals.get()) {
|
||||
for (Interval dataInterval : dataGranularity.getIterable(segmentInterval)) {
|
||||
intervals.add(dataInterval);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -25,6 +25,8 @@ import com.google.common.collect.Lists;
|
|||
import io.druid.db.DbConnectorConfig;
|
||||
import io.druid.indexer.granularity.UniformGranularitySpec;
|
||||
import io.druid.indexer.partitions.PartitionsSpec;
|
||||
import io.druid.indexer.partitions.RandomPartitionsSpec;
|
||||
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||
import io.druid.indexer.updater.DbUpdaterJobSpec;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
|
@ -65,7 +67,7 @@ public class HadoopDruidIndexerConfigTest
|
|||
Assert.assertEquals(
|
||||
"getIntervals",
|
||||
Lists.newArrayList(new Interval("2012-01-01/P1D")),
|
||||
granularitySpec.getIntervals()
|
||||
granularitySpec.getIntervals().get()
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
|
@ -99,7 +101,7 @@ public class HadoopDruidIndexerConfigTest
|
|||
Assert.assertEquals(
|
||||
"getIntervals",
|
||||
Lists.newArrayList(new Interval("2012-02-01/P1D")),
|
||||
granularitySpec.getIntervals()
|
||||
granularitySpec.getIntervals().get()
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
|
@ -167,15 +169,14 @@ public class HadoopDruidIndexerConfigTest
|
|||
100
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"getPartitionDimension",
|
||||
partitionsSpec.getPartitionDimension(),
|
||||
null
|
||||
Assert.assertTrue(
|
||||
"partitionSpec",
|
||||
partitionsSpec instanceof SingleDimensionPartitionsSpec
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPartitionsSpecSpecificDimension()
|
||||
public void testPartitionsSpecSpecificDimensionLegacy()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
|
@ -214,9 +215,10 @@ public class HadoopDruidIndexerConfigTest
|
|||
150
|
||||
);
|
||||
|
||||
Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof SingleDimensionPartitionsSpec);
|
||||
Assert.assertEquals(
|
||||
"getPartitionDimension",
|
||||
partitionsSpec.getPartitionDimension(),
|
||||
((SingleDimensionPartitionsSpec)partitionsSpec).getPartitionDimension(),
|
||||
"foo"
|
||||
);
|
||||
}
|
||||
|
@ -259,9 +261,10 @@ public class HadoopDruidIndexerConfigTest
|
|||
150
|
||||
);
|
||||
|
||||
Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof SingleDimensionPartitionsSpec);
|
||||
Assert.assertEquals(
|
||||
"getPartitionDimension",
|
||||
partitionsSpec.getPartitionDimension(),
|
||||
((SingleDimensionPartitionsSpec)partitionsSpec).getPartitionDimension(),
|
||||
"foo"
|
||||
);
|
||||
}
|
||||
|
@ -307,9 +310,10 @@ public class HadoopDruidIndexerConfigTest
|
|||
200
|
||||
);
|
||||
|
||||
Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof SingleDimensionPartitionsSpec);
|
||||
Assert.assertEquals(
|
||||
"getPartitionDimension",
|
||||
partitionsSpec.getPartitionDimension(),
|
||||
((SingleDimensionPartitionsSpec)partitionsSpec).getPartitionDimension(),
|
||||
"foo"
|
||||
);
|
||||
}
|
||||
|
@ -420,7 +424,6 @@ public class HadoopDruidIndexerConfigTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void shouldMakeHDFSCompliantSegmentOutputPath()
|
||||
{
|
||||
|
@ -497,4 +500,47 @@ public class HadoopDruidIndexerConfigTest
|
|||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public void testRandomPartitionsSpec() throws Exception{
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
cfg = jsonReadWriteRead(
|
||||
"{"
|
||||
+ "\"partitionsSpec\":{"
|
||||
+ " \"targetPartitionSize\":100,"
|
||||
+ " \"type\":\"random\""
|
||||
+ " }"
|
||||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
final PartitionsSpec partitionsSpec = cfg.getPartitionsSpec();
|
||||
|
||||
Assert.assertEquals(
|
||||
"isDeterminingPartitions",
|
||||
partitionsSpec.isDeterminingPartitions(),
|
||||
true
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"getTargetPartitionSize",
|
||||
partitionsSpec.getTargetPartitionSize(),
|
||||
100
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"getMaxPartitionSize",
|
||||
partitionsSpec.getMaxPartitionSize(),
|
||||
150
|
||||
);
|
||||
|
||||
Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof RandomPartitionsSpec);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,7 +54,7 @@ public class ArbitraryGranularityTest
|
|||
new Interval("2012-01-08T00Z/2012-01-11T00Z"),
|
||||
new Interval("2012-02-01T00Z/2012-03-01T00Z")
|
||||
),
|
||||
Lists.newArrayList(spec.bucketIntervals())
|
||||
Lists.newArrayList(spec.bucketIntervals().get())
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
|
|
|
@ -57,7 +57,7 @@ public class UniformGranularityTest
|
|||
new Interval("2012-01-09T00Z/P1D"),
|
||||
new Interval("2012-01-10T00Z/P1D")
|
||||
),
|
||||
Lists.newArrayList(spec.bucketIntervals())
|
||||
Lists.newArrayList(spec.bucketIntervals().get())
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -52,7 +52,7 @@ import java.util.concurrent.TimeUnit;
|
|||
* firehoses with an {@link ServiceAnnouncingChatHandlerProvider}.
|
||||
*/
|
||||
@JsonTypeName("receiver")
|
||||
public class EventReceiverFirehoseFactory implements FirehoseFactory
|
||||
public class EventReceiverFirehoseFactory implements FirehoseFactory<MapInputRowParser>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(EventReceiverFirehoseFactory.class);
|
||||
private static final int DEFAULT_BUFFER_SIZE = 100000;
|
||||
|
@ -77,11 +77,11 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(MapInputRowParser firehoseParser) throws IOException
|
||||
{
|
||||
log.info("Connecting firehose: %s", serviceName);
|
||||
|
||||
final EventReceiverFirehose firehose = new EventReceiverFirehose();
|
||||
final EventReceiverFirehose firehose = new EventReceiverFirehose(firehoseParser);
|
||||
|
||||
if (chatHandlerProvider.isPresent()) {
|
||||
log.info("Found chathandler of class[%s]", chatHandlerProvider.get().getClass().getName());
|
||||
|
@ -117,15 +117,17 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory
|
|||
public class EventReceiverFirehose implements ChatHandler, Firehose
|
||||
{
|
||||
private final BlockingQueue<InputRow> buffer;
|
||||
private final MapInputRowParser parser;
|
||||
|
||||
private final Object readLock = new Object();
|
||||
|
||||
private volatile InputRow nextRow = null;
|
||||
private volatile boolean closed = false;
|
||||
|
||||
public EventReceiverFirehose()
|
||||
public EventReceiverFirehose(MapInputRowParser parser)
|
||||
{
|
||||
this.buffer = new ArrayBlockingQueue<InputRow>(bufferSize);
|
||||
this.parser = parser;
|
||||
}
|
||||
|
||||
@POST
|
||||
|
|
|
@ -37,6 +37,7 @@ import io.druid.segment.IndexMerger;
|
|||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.SegmentGranularity;
|
||||
import io.druid.segment.SegmentUtils;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
import io.druid.segment.loading.DataSegmentPusher;
|
||||
import io.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import io.druid.segment.realtime.FireHydrant;
|
||||
|
@ -87,7 +88,7 @@ public class YeOldePlumberSchool implements PlumberSchool
|
|||
}
|
||||
|
||||
@Override
|
||||
public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics)
|
||||
public Plumber findPlumber(final DataSchema schema, final FireDepartmentMetrics metrics)
|
||||
{
|
||||
// There can be only one.
|
||||
final Sink theSink = new Sink(interval, schema, version);
|
||||
|
|
|
@ -24,25 +24,23 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.api.client.util.Lists;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.google.common.collect.Multimaps;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.indexer.HadoopDruidDetermineConfigurationJob;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfig;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfigBuilder;
|
||||
import io.druid.indexer.HadoopDruidIndexerJob;
|
||||
import io.druid.indexer.HadoopDruidIndexerSchema;
|
||||
import io.druid.indexer.Jobby;
|
||||
import io.druid.indexing.common.TaskLock;
|
||||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.indexing.common.TaskToolbox;
|
||||
import io.druid.indexing.common.actions.LockAcquireAction;
|
||||
import io.druid.indexing.common.actions.LockTryAcquireAction;
|
||||
import io.druid.indexing.common.actions.SegmentInsertAction;
|
||||
import io.druid.indexing.common.actions.TaskActionClient;
|
||||
import io.druid.initialization.Initialization;
|
||||
import io.druid.server.initialization.ExtensionsConfig;
|
||||
|
@ -51,30 +49,26 @@ import io.tesla.aether.internal.DefaultTeslaAether;
|
|||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.lang.reflect.Method;
|
||||
import java.net.URL;
|
||||
import java.net.URLClassLoader;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.SortedSet;
|
||||
|
||||
public class HadoopIndexTask extends AbstractFixedIntervalTask
|
||||
public class HadoopIndexTask extends AbstractTask
|
||||
{
|
||||
private static final Logger log = new Logger(HadoopIndexTask.class);
|
||||
private static String defaultHadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3";
|
||||
|
||||
private static final ExtensionsConfig extensionsConfig;
|
||||
|
||||
static {
|
||||
extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class);
|
||||
}
|
||||
|
||||
private static String defaultHadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3";
|
||||
@JsonIgnore
|
||||
private final HadoopDruidIndexerSchema schema;
|
||||
|
||||
@JsonIgnore
|
||||
private final String hadoopCoordinates;
|
||||
|
||||
|
@ -97,13 +91,7 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask
|
|||
{
|
||||
super(
|
||||
id != null ? id : String.format("index_hadoop_%s_%s", schema.getDataSource(), new DateTime()),
|
||||
schema.getDataSource(),
|
||||
JodaUtils.umbrellaInterval(
|
||||
JodaUtils.condenseIntervals(
|
||||
schema.getGranularitySpec()
|
||||
.bucketIntervals()
|
||||
)
|
||||
)
|
||||
schema.getDataSource()
|
||||
);
|
||||
|
||||
// Some HadoopDruidIndexerSchema stuff doesn't make sense in the context of the indexing service
|
||||
|
@ -121,6 +109,22 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask
|
|||
return "index_hadoop";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReady(TaskActionClient taskActionClient) throws Exception
|
||||
{
|
||||
Optional<SortedSet<Interval>> intervals = schema.getGranularitySpec().bucketIntervals();
|
||||
if (intervals.isPresent()) {
|
||||
Interval interval = JodaUtils.umbrellaInterval(
|
||||
JodaUtils.condenseIntervals(
|
||||
intervals.get()
|
||||
)
|
||||
);
|
||||
return taskActionClient.submit(new LockTryAcquireAction(interval)).isPresent();
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty("config")
|
||||
public HadoopDruidIndexerSchema getSchema()
|
||||
{
|
||||
|
@ -167,29 +171,60 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask
|
|||
jobUrls.addAll(extensionURLs);
|
||||
|
||||
System.setProperty("druid.hadoop.internal.classpath", Joiner.on(File.pathSeparator).join(jobUrls));
|
||||
boolean determineIntervals = !schema.getGranularitySpec().bucketIntervals().isPresent();
|
||||
|
||||
final Class<?> mainClass = loader.loadClass(HadoopIndexTaskInnerProcessing.class.getName());
|
||||
final Method mainMethod = mainClass.getMethod("runTask", String[].class);
|
||||
final Class<?> determineConfigurationMainClass = loader.loadClass(HadoopDetermineConfigInnerProcessing.class.getName());
|
||||
final Method determineConfigurationMainMethod = determineConfigurationMainClass.getMethod(
|
||||
"runTask",
|
||||
String[].class
|
||||
);
|
||||
|
||||
// We should have a lock from before we started running
|
||||
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
|
||||
log.info("Setting version to: %s", myLock.getVersion());
|
||||
|
||||
String[] args = new String[]{
|
||||
String[] determineConfigArgs = new String[]{
|
||||
toolbox.getObjectMapper().writeValueAsString(schema),
|
||||
myLock.getVersion(),
|
||||
toolbox.getConfig().getHadoopWorkingPath(),
|
||||
toolbox.getSegmentPusher().getPathForHadoop(getDataSource()),
|
||||
toolbox.getSegmentPusher().getPathForHadoop(getDataSource())
|
||||
};
|
||||
|
||||
String segments = (String) mainMethod.invoke(null, new Object[]{args});
|
||||
String config = (String) determineConfigurationMainMethod.invoke(null, new Object[]{determineConfigArgs});
|
||||
HadoopDruidIndexerSchema indexerSchema = toolbox.getObjectMapper()
|
||||
.readValue(config, HadoopDruidIndexerSchema.class);
|
||||
|
||||
|
||||
// We should have a lock from before we started running only if interval was specified
|
||||
final String version;
|
||||
if (determineIntervals) {
|
||||
Interval interval = JodaUtils.umbrellaInterval(
|
||||
JodaUtils.condenseIntervals(
|
||||
indexerSchema.getGranularitySpec().bucketIntervals().get()
|
||||
)
|
||||
);
|
||||
TaskLock lock = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval));
|
||||
version = lock.getVersion();
|
||||
} else {
|
||||
Iterable<TaskLock> locks = getTaskLocks(toolbox);
|
||||
final TaskLock myLock = Iterables.getOnlyElement(locks);
|
||||
version = myLock.getVersion();
|
||||
}
|
||||
log.info("Setting version to: %s", version);
|
||||
|
||||
final Class<?> indexGeneratorMainClass = loader.loadClass(HadoopIndexGeneratorInnerProcessing.class.getName());
|
||||
final Method indexGeneratorMainMethod = indexGeneratorMainClass.getMethod("runTask", String[].class);
|
||||
String[] indexGeneratorArgs = new String[]{
|
||||
toolbox.getObjectMapper().writeValueAsString(indexerSchema),
|
||||
version
|
||||
};
|
||||
String segments = (String) indexGeneratorMainMethod.invoke(null, new Object[]{indexGeneratorArgs});
|
||||
|
||||
|
||||
if (segments != null) {
|
||||
|
||||
List<DataSegment> publishedSegments = toolbox.getObjectMapper().readValue(
|
||||
segments,
|
||||
new TypeReference<List<DataSegment>>() {}
|
||||
new TypeReference<List<DataSegment>>()
|
||||
{
|
||||
}
|
||||
);
|
||||
|
||||
toolbox.pushSegments(publishedSegments);
|
||||
return TaskStatus.success(getId());
|
||||
} else {
|
||||
|
@ -197,14 +232,12 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask
|
|||
}
|
||||
}
|
||||
|
||||
public static class HadoopIndexTaskInnerProcessing
|
||||
public static class HadoopIndexGeneratorInnerProcessing
|
||||
{
|
||||
public static String runTask(String[] args) throws Exception
|
||||
{
|
||||
final String schema = args[0];
|
||||
final String version = args[1];
|
||||
final String workingPath = args[2];
|
||||
final String segmentOutputPath = args[3];
|
||||
String version = args[1];
|
||||
|
||||
final HadoopDruidIndexerSchema theSchema = HadoopDruidIndexerConfig.jsonMapper
|
||||
.readValue(
|
||||
|
@ -214,12 +247,6 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask
|
|||
final HadoopDruidIndexerConfig config =
|
||||
new HadoopDruidIndexerConfigBuilder().withSchema(theSchema)
|
||||
.withVersion(version)
|
||||
.withWorkingPath(
|
||||
workingPath
|
||||
)
|
||||
.withSegmentOutputPath(
|
||||
segmentOutputPath
|
||||
)
|
||||
.build();
|
||||
|
||||
HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(config);
|
||||
|
@ -232,4 +259,34 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask
|
|||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
public static class HadoopDetermineConfigInnerProcessing
|
||||
{
|
||||
public static String runTask(String[] args) throws Exception
|
||||
{
|
||||
final String schema = args[0];
|
||||
final String workingPath = args[1];
|
||||
final String segmentOutputPath = args[2];
|
||||
|
||||
final HadoopDruidIndexerSchema theSchema = HadoopDruidIndexerConfig.jsonMapper
|
||||
.readValue(
|
||||
schema,
|
||||
HadoopDruidIndexerSchema.class
|
||||
);
|
||||
final HadoopDruidIndexerConfig config =
|
||||
new HadoopDruidIndexerConfigBuilder().withSchema(theSchema)
|
||||
.withWorkingPath(workingPath)
|
||||
.withSegmentOutputPath(segmentOutputPath)
|
||||
.build();
|
||||
|
||||
Jobby job = new HadoopDruidDetermineConfigurationJob(config);
|
||||
|
||||
log.info("Starting a hadoop index generator job...");
|
||||
if (job.run()) {
|
||||
return HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(HadoopDruidIndexerConfigBuilder.toSchema(config));
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ import io.druid.indexing.common.TaskToolbox;
|
|||
import io.druid.indexing.common.actions.SegmentInsertAction;
|
||||
import io.druid.indexing.common.index.YeOldePlumberSchool;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
import io.druid.segment.loading.DataSegmentPusher;
|
||||
import io.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import io.druid.segment.realtime.Schema;
|
||||
|
@ -109,8 +110,8 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
id != null ? id : String.format("index_%s_%s", dataSource, new DateTime().toString()),
|
||||
dataSource,
|
||||
new Interval(
|
||||
granularitySpec.bucketIntervals().first().getStart(),
|
||||
granularitySpec.bucketIntervals().last().getEnd()
|
||||
granularitySpec.bucketIntervals().get().first().getStart(),
|
||||
granularitySpec.bucketIntervals().get().last().getEnd()
|
||||
)
|
||||
);
|
||||
|
||||
|
@ -137,7 +138,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
|
||||
final Set<DataSegment> segments = Sets.newHashSet();
|
||||
|
||||
final Set<Interval> validIntervals = Sets.intersection(granularitySpec.bucketIntervals(), getDataIntervals());
|
||||
final Set<Interval> validIntervals = Sets.intersection(granularitySpec.bucketIntervals().get(), getDataIntervals());
|
||||
if (validIntervals.isEmpty()) {
|
||||
throw new ISE("No valid data intervals found. Check your configs!");
|
||||
}
|
||||
|
@ -152,11 +153,11 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
for (final ShardSpec shardSpec : shardSpecs) {
|
||||
final DataSegment segment = generateSegment(
|
||||
toolbox,
|
||||
new Schema(
|
||||
new DataSchema(
|
||||
getDataSource(),
|
||||
spatialDimensions,
|
||||
firehoseFactory.getParser(),
|
||||
aggregators,
|
||||
indexGranularity,
|
||||
new io.druid.segment.indexing.GranularitySpec(null, indexGranularity),
|
||||
shardSpec
|
||||
),
|
||||
bucket,
|
||||
|
@ -172,7 +173,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
private SortedSet<Interval> getDataIntervals() throws IOException
|
||||
{
|
||||
SortedSet<Interval> retVal = Sets.newTreeSet(Comparators.intervalsByStartThenEnd());
|
||||
try (Firehose firehose = firehoseFactory.connect()) {
|
||||
try (Firehose firehose = firehoseFactory.connect(firehoseFactory.getParser())) {
|
||||
while (firehose.hasMore()) {
|
||||
final InputRow inputRow = firehose.nextRow();
|
||||
Interval interval = granularitySpec.getGranularity().bucket(new DateTime(inputRow.getTimestampFromEpoch()));
|
||||
|
@ -197,7 +198,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
final Map<String, TreeMultiset<String>> dimensionValueMultisets = Maps.newHashMap();
|
||||
|
||||
// Load data
|
||||
try (Firehose firehose = firehoseFactory.connect()) {
|
||||
try (Firehose firehose = firehoseFactory.connect(firehoseFactory.getParser())) {
|
||||
while (firehose.hasMore()) {
|
||||
final InputRow inputRow = firehose.nextRow();
|
||||
if (interval.contains(inputRow.getTimestampFromEpoch())) {
|
||||
|
@ -305,7 +306,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
|
||||
private DataSegment generateSegment(
|
||||
final TaskToolbox toolbox,
|
||||
final Schema schema,
|
||||
final DataSchema schema,
|
||||
final Interval interval,
|
||||
final String version
|
||||
) throws IOException
|
||||
|
@ -344,7 +345,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
|
||||
// Create firehose + plumber
|
||||
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
|
||||
final Firehose firehose = firehoseFactory.connect();
|
||||
final Firehose firehose = firehoseFactory.connect(firehoseFactory.getParser());
|
||||
final Plumber plumber = new YeOldePlumberSchool(
|
||||
interval,
|
||||
version,
|
||||
|
@ -420,7 +421,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
*
|
||||
* @return true or false
|
||||
*/
|
||||
private boolean shouldIndex(final Schema schema, final Interval interval, final InputRow inputRow)
|
||||
private boolean shouldIndex(final DataSchema schema, final Interval interval, final InputRow inputRow)
|
||||
{
|
||||
return interval.contains(inputRow.getTimestampFromEpoch()) && schema.getShardSpec().isInChunk(inputRow);
|
||||
}
|
||||
|
|
|
@ -130,7 +130,7 @@ public class NoopTask extends AbstractTask
|
|||
{
|
||||
if (firehoseFactory != null) {
|
||||
log.info("Connecting firehose");
|
||||
firehoseFactory.connect();
|
||||
firehoseFactory.connect(firehoseFactory.getParser());
|
||||
}
|
||||
|
||||
log.info("Running noop task[%s]", getId());
|
||||
|
|
|
@ -43,6 +43,11 @@ import io.druid.query.QueryRunnerFactory;
|
|||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.segment.SegmentGranularity;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
import io.druid.segment.indexing.DriverConfig;
|
||||
import io.druid.segment.indexing.GranularitySpec;
|
||||
import io.druid.segment.indexing.RealtimeDriverConfig;
|
||||
import io.druid.segment.indexing.RealtimeIOConfig;
|
||||
import io.druid.segment.realtime.FireDepartment;
|
||||
import io.druid.segment.realtime.FireDepartmentConfig;
|
||||
import io.druid.segment.realtime.RealtimeMetricsMonitor;
|
||||
|
@ -194,7 +199,7 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
|
||||
// Set up firehose
|
||||
final Period intermediatePersistPeriod = fireDepartmentConfig.getIntermediatePersistPeriod();
|
||||
final Firehose firehose = firehoseFactory.connect();
|
||||
final Firehose firehose = firehoseFactory.connect(firehoseFactory.getParser());
|
||||
|
||||
// It would be nice to get the PlumberSchool in the constructor. Although that will need jackson injectables for
|
||||
// stuff like the ServerView, which seems kind of odd? Perhaps revisit this when Guice has been introduced.
|
||||
|
@ -298,10 +303,23 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
realtimePlumberSchool.setRejectionPolicyFactory(rejectionPolicyFactory);
|
||||
}
|
||||
|
||||
final FireDepartment fireDepartment = new FireDepartment(schema, fireDepartmentConfig, null, null);
|
||||
DataSchema dataSchema = new DataSchema(
|
||||
schema.getDataSource(),
|
||||
firehoseFactory.getParser(),
|
||||
schema.getAggregators(),
|
||||
new GranularitySpec(realtimePlumberSchool.getSegmentGranularity(), schema.getIndexGranularity()),
|
||||
schema.getShardSpec()
|
||||
);
|
||||
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(firehoseFactory, realtimePlumberSchool);
|
||||
RealtimeDriverConfig driverConfig = new RealtimeDriverConfig(
|
||||
fireDepartmentConfig.getMaxRowsInMemory(),
|
||||
fireDepartmentConfig.getIntermediatePersistPeriod()
|
||||
);
|
||||
|
||||
final FireDepartment fireDepartment = new FireDepartment(dataSchema, realtimeIOConfig, driverConfig, null, null, null, null);
|
||||
final RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor(ImmutableList.of(fireDepartment));
|
||||
this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate();
|
||||
this.plumber = realtimePlumberSchool.findPlumber(schema, fireDepartment.getMetrics());
|
||||
this.plumber = realtimePlumberSchool.findPlumber(dataSchema, fireDepartment.getMetrics());
|
||||
|
||||
try {
|
||||
plumber.startJob();
|
||||
|
|
|
@ -297,22 +297,29 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
|
|||
createValidWorkerPredicate(config, workerSetupData)
|
||||
);
|
||||
final Predicate<ZkWorker> isLazyWorker = createLazyWorkerPredicate(config, workerSetupData);
|
||||
final int minWorkerCount = workerSetupData.getMinNumWorkers();
|
||||
final int maxWorkerCount = workerSetupData.getMaxNumWorkers();
|
||||
|
||||
if (minWorkerCount > maxWorkerCount) {
|
||||
log.error("Huh? minWorkerCount[%d] > maxWorkerCount[%d]. I give up!", minWorkerCount, maxWorkerCount);
|
||||
return;
|
||||
}
|
||||
|
||||
if (targetWorkerCount < 0) {
|
||||
// Initialize to size of current worker pool, subject to pool size limits
|
||||
targetWorkerCount = Math.max(
|
||||
Math.min(
|
||||
zkWorkers.size(),
|
||||
workerSetupData.getMaxNumWorkers()
|
||||
maxWorkerCount
|
||||
),
|
||||
workerSetupData.getMinNumWorkers()
|
||||
minWorkerCount
|
||||
);
|
||||
log.info(
|
||||
"Starting with a target of %,d workers (current = %,d, min = %,d, max = %,d).",
|
||||
targetWorkerCount,
|
||||
validWorkers.size(),
|
||||
workerSetupData.getMinNumWorkers(),
|
||||
workerSetupData.getMaxNumWorkers()
|
||||
minWorkerCount,
|
||||
maxWorkerCount
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -320,36 +327,37 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
|
|||
&& currentlyTerminating.isEmpty()
|
||||
&& validWorkers.size() == targetWorkerCount;
|
||||
final boolean shouldScaleUp = atSteadyState
|
||||
&& hasTaskPendingBeyondThreshold(pendingTasks)
|
||||
&& targetWorkerCount < workerSetupData.getMaxNumWorkers();
|
||||
&& targetWorkerCount < maxWorkerCount
|
||||
&& (hasTaskPendingBeyondThreshold(pendingTasks)
|
||||
|| targetWorkerCount < minWorkerCount);
|
||||
final boolean shouldScaleDown = atSteadyState
|
||||
&& Iterables.any(validWorkers, isLazyWorker)
|
||||
&& targetWorkerCount > workerSetupData.getMinNumWorkers();
|
||||
&& targetWorkerCount > minWorkerCount
|
||||
&& Iterables.any(validWorkers, isLazyWorker);
|
||||
if (shouldScaleUp) {
|
||||
targetWorkerCount++;
|
||||
targetWorkerCount = Math.max(targetWorkerCount + 1, minWorkerCount);
|
||||
log.info(
|
||||
"I think we should scale up to %,d workers (current = %,d, min = %,d, max = %,d).",
|
||||
targetWorkerCount,
|
||||
validWorkers.size(),
|
||||
workerSetupData.getMinNumWorkers(),
|
||||
workerSetupData.getMaxNumWorkers()
|
||||
minWorkerCount,
|
||||
maxWorkerCount
|
||||
);
|
||||
} else if (shouldScaleDown) {
|
||||
targetWorkerCount--;
|
||||
targetWorkerCount = Math.min(targetWorkerCount - 1, maxWorkerCount);
|
||||
log.info(
|
||||
"I think we should scale down to %,d workers (current = %,d, min = %,d, max = %,d).",
|
||||
targetWorkerCount,
|
||||
validWorkers.size(),
|
||||
workerSetupData.getMinNumWorkers(),
|
||||
workerSetupData.getMaxNumWorkers()
|
||||
minWorkerCount,
|
||||
maxWorkerCount
|
||||
);
|
||||
} else {
|
||||
log.info(
|
||||
"Our target is %,d workers, and I'm okay with that (current = %,d, min = %,d, max = %,d).",
|
||||
targetWorkerCount,
|
||||
validWorkers.size(),
|
||||
workerSetupData.getMinNumWorkers(),
|
||||
workerSetupData.getMaxNumWorkers()
|
||||
minWorkerCount,
|
||||
maxWorkerCount
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -382,11 +382,9 @@ public class TaskSerdeTest
|
|||
final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.readValue(json, Task.class);
|
||||
|
||||
Assert.assertEquals("foo", task.getDataSource());
|
||||
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getInterval(), task2.getInterval());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,6 +41,7 @@ import io.druid.data.input.Firehose;
|
|||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.indexer.granularity.UniformGranularitySpec;
|
||||
import io.druid.indexing.common.SegmentLoaderFactory;
|
||||
|
@ -586,7 +587,7 @@ public class TaskLifecycleTest
|
|||
return new FirehoseFactory()
|
||||
{
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(InputRowParser parser) throws IOException
|
||||
{
|
||||
return new Firehose()
|
||||
{
|
||||
|
@ -622,6 +623,12 @@ public class TaskLifecycleTest
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser getParser()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -630,7 +637,7 @@ public class TaskLifecycleTest
|
|||
return new FirehoseFactory()
|
||||
{
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(InputRowParser parser) throws IOException
|
||||
{
|
||||
final Iterator<InputRow> inputRowIterator = inputRows.iterator();
|
||||
|
||||
|
@ -668,6 +675,12 @@ public class TaskLifecycleTest
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser getParser()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -349,6 +349,61 @@ public class SimpleResourceManagementStrategyTest
|
|||
EasyMock.verify(autoScalingStrategy);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMinCountIncrease() throws Exception
|
||||
{
|
||||
// Don't terminate anything
|
||||
EasyMock.reset(autoScalingStrategy);
|
||||
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
|
||||
.andReturn(Lists.<String>newArrayList("ip"));
|
||||
EasyMock.replay(autoScalingStrategy);
|
||||
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(NoopTask.create(), "h1", "i1", "0")
|
||||
)
|
||||
);
|
||||
Assert.assertFalse(terminatedSomething);
|
||||
EasyMock.verify(autoScalingStrategy);
|
||||
|
||||
// Don't provision anything
|
||||
EasyMock.reset(autoScalingStrategy);
|
||||
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
|
||||
.andReturn(Lists.<String>newArrayList("ip"));
|
||||
EasyMock.replay(autoScalingStrategy);
|
||||
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(NoopTask.create())
|
||||
)
|
||||
);
|
||||
Assert.assertFalse(provisionedSomething);
|
||||
EasyMock.verify(autoScalingStrategy);
|
||||
|
||||
// Increase minNumWorkers
|
||||
workerSetupData.set(new WorkerSetupData("0", 3, 5, null, null, null));
|
||||
|
||||
// Should provision two new workers
|
||||
EasyMock.reset(autoScalingStrategy);
|
||||
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
|
||||
.andReturn(Lists.<String>newArrayList("ip"));
|
||||
EasyMock.expect(autoScalingStrategy.provision()).andReturn(
|
||||
new AutoScalingData(Lists.<String>newArrayList("h3"))
|
||||
);
|
||||
EasyMock.expect(autoScalingStrategy.provision()).andReturn(
|
||||
new AutoScalingData(Lists.<String>newArrayList("h4"))
|
||||
);
|
||||
EasyMock.replay(autoScalingStrategy);
|
||||
provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(NoopTask.create(), "h1", "i1", "0")
|
||||
)
|
||||
);
|
||||
Assert.assertTrue(provisionedSomething);
|
||||
EasyMock.verify(autoScalingStrategy);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMinVersionIncrease() throws Exception
|
||||
{
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -23,12 +23,14 @@ package io.druid.firehose.kafka;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.ByteBufferInputRowParser;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.StringInputRowParser;
|
||||
import kafka.consumer.Consumer;
|
||||
import kafka.consumer.ConsumerConfig;
|
||||
import kafka.consumer.ConsumerIterator;
|
||||
|
@ -40,10 +42,11 @@ import java.nio.ByteBuffer;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class KafkaEightFirehoseFactory implements FirehoseFactory
|
||||
public class KafkaEightFirehoseFactory implements FirehoseFactory<ByteBufferInputRowParser>
|
||||
{
|
||||
private static final Logger log = new Logger(KafkaEightFirehoseFactory.class);
|
||||
|
||||
|
@ -65,13 +68,24 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory
|
|||
{
|
||||
this.consumerProps = consumerProps;
|
||||
this.feed = feed;
|
||||
this.parser = parser;
|
||||
|
||||
parser.addDimensionExclusion("feed");
|
||||
Set<String> newDimExclus = Sets.union(
|
||||
parser.getParseSpec().getDimensionsSpec().getDimensionExclusions(),
|
||||
Sets.newHashSet("feed")
|
||||
);
|
||||
this.parser = parser.withParseSpec(
|
||||
parser.getParseSpec()
|
||||
.withDimensionsSpec(
|
||||
parser.getParseSpec()
|
||||
.getDimensionsSpec()
|
||||
.withDimensionExclusions(
|
||||
newDimExclus
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(final ByteBufferInputRowParser firehoseParser) throws IOException
|
||||
{
|
||||
final ConsumerConnector connector = Consumer.createJavaConsumerConnector(new ConsumerConfig(consumerProps));
|
||||
|
||||
|
@ -108,7 +122,7 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
try {
|
||||
return parser.parse(ByteBuffer.wrap(message));
|
||||
return firehoseParser.parse(ByteBuffer.wrap(message));
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new FormattedException.Builder()
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.firehose.kafka;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.ByteBufferInputRowParser;
|
||||
|
@ -40,10 +41,11 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class KafkaSevenFirehoseFactory implements FirehoseFactory
|
||||
public class KafkaSevenFirehoseFactory implements FirehoseFactory<ByteBufferInputRowParser>
|
||||
{
|
||||
private static final Logger log = new Logger(KafkaSevenFirehoseFactory.class);
|
||||
|
||||
|
@ -60,9 +62,20 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory
|
|||
{
|
||||
this.consumerProps = consumerProps;
|
||||
this.feed = feed;
|
||||
this.parser = parser;
|
||||
|
||||
parser.addDimensionExclusion("feed");
|
||||
Set<String> newDimExclus = Sets.union(
|
||||
parser.getParseSpec().getDimensionsSpec().getDimensionExclusions(),
|
||||
Sets.newHashSet("feed")
|
||||
);
|
||||
this.parser = parser.withParseSpec(
|
||||
parser.getParseSpec()
|
||||
.withDimensionsSpec(
|
||||
parser.getParseSpec()
|
||||
.getDimensionsSpec()
|
||||
.withDimensionExclusions(
|
||||
newDimExclus
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -84,7 +97,7 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(final ByteBufferInputRowParser firehoseParser) throws IOException
|
||||
{
|
||||
final ConsumerConnector connector = Consumer.createJavaConsumerConnector(new ConsumerConfig(consumerProps));
|
||||
|
||||
|
@ -121,7 +134,7 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory
|
|||
public InputRow parseMessage(Message message) throws FormattedException
|
||||
{
|
||||
try {
|
||||
return parser.parse(message.payload());
|
||||
return firehoseParser.parse(message.payload());
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new FormattedException.Builder()
|
||||
|
|
6
pom.xml
6
pom.xml
|
@ -23,7 +23,7 @@
|
|||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<packaging>pom</packaging>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
<name>druid</name>
|
||||
<description>druid</description>
|
||||
<scm>
|
||||
|
@ -39,8 +39,8 @@
|
|||
|
||||
<properties>
|
||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
<metamx.java-util.version>0.25.2</metamx.java-util.version>
|
||||
<apache.curator.version>2.3.0</apache.curator.version>
|
||||
<metamx.java-util.version>0.25.3</metamx.java-util.version>
|
||||
<apache.curator.version>2.4.0</apache.curator.version>
|
||||
<druid.api.version>0.2.0-SNAPSHOT</druid.api.version>
|
||||
</properties>
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -87,7 +87,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser
|
|||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser withParseSpec(ParseSpec parseSpec)
|
||||
public ProtoBufInputRowParser withParseSpec(ParseSpec parseSpec)
|
||||
{
|
||||
return new ProtoBufInputRowParser(parseSpec, descriptorFileInClasspath, null, null, null, null);
|
||||
}
|
||||
|
|
|
@ -22,7 +22,9 @@ package io.druid.query;
|
|||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
|
||||
public class MetricsEmittingExecutorService extends DelegatingExecutorService
|
||||
|
@ -44,6 +46,13 @@ public class MetricsEmittingExecutorService extends DelegatingExecutorService
|
|||
this.metricBuilder = metricBuilder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> Future<T> submit(Callable<T> tCallable)
|
||||
{
|
||||
emitMetrics();
|
||||
return base.submit(tCallable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(Runnable runnable)
|
||||
{
|
||||
|
@ -53,8 +62,8 @@ public class MetricsEmittingExecutorService extends DelegatingExecutorService
|
|||
|
||||
private void emitMetrics()
|
||||
{
|
||||
if (base instanceof ThreadPoolExecutor) {
|
||||
emitter.emit(metricBuilder.build("exec/backlog", ((ThreadPoolExecutor) base).getQueue().size()));
|
||||
if (base instanceof PrioritizedExecutorService) {
|
||||
emitter.emit(metricBuilder.build("exec/backlog", ((PrioritizedExecutorService) base).getQueueSize()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -68,6 +68,7 @@ public class PrioritizedExecutorService extends AbstractExecutorService
|
|||
|
||||
return service;
|
||||
}
|
||||
|
||||
private static final int DEFAULT_PRIORITY = 0;
|
||||
|
||||
|
||||
|
@ -133,6 +134,11 @@ public class PrioritizedExecutorService extends AbstractExecutorService
|
|||
return new PrioritizedFuture<T>((PrioritizedCallable) theCallable);
|
||||
}
|
||||
|
||||
public int getQueueSize()
|
||||
{
|
||||
return threadPoolExecutor.getQueue().size();
|
||||
}
|
||||
|
||||
private static class PrioritizedFuture<V> extends FutureTask<V> implements Comparable<PrioritizedFuture>
|
||||
{
|
||||
private final PrioritizedCallable<V> callable;
|
||||
|
|
|
@ -166,9 +166,9 @@ public class TestIndex
|
|||
StringInputRowParser parser = new StringInputRowParser(
|
||||
new DelimitedParseSpec(
|
||||
new TimestampSpec("ts", "iso"),
|
||||
new DimensionsSpec(Arrays.asList(COLUMNS), Arrays.asList(DIMENSIONS), null),
|
||||
new DimensionsSpec(Arrays.asList(DIMENSIONS), null, null),
|
||||
"\t",
|
||||
Arrays.<String>asList()
|
||||
Arrays.asList(COLUMNS)
|
||||
),
|
||||
null, null, null, null
|
||||
);
|
||||
|
|
Binary file not shown.
|
@ -206,7 +206,7 @@ for queries on events that exist in this JVM heap-based buffer. To avoid heap ov
|
|||
problems, real-time nodes persist their in-memory indexes to disk either
|
||||
periodically or after some maximum row limit is reached. This persist process
|
||||
converts data stored in the in-memory buffer to a column oriented storage
|
||||
format described in \ref{sec:storage-format}. Each persisted index is immutable and
|
||||
format described in Section \ref{sec:storage-format}. Each persisted index is immutable and
|
||||
real-time nodes load persisted indexes into off-heap memory such that they can
|
||||
still be queried. Figure~\ref{fig:realtime_flow} illustrates the process.
|
||||
|
||||
|
@ -533,7 +533,7 @@ contain strings. Storing strings directly is unnecessarily costly and string
|
|||
columns can be dictionary encoded instead. Dictionary encoding is a common
|
||||
method to compress data and has been used in other data stores such as
|
||||
PowerDrill \cite{hall2012processing}. In the example in
|
||||
Table~\ref{tab:sample_data}, we can map each publisher to an unique integer
|
||||
Table~\ref{tab:sample_data}, we can map each page to an unique integer
|
||||
identifier.
|
||||
\begin{verbatim}
|
||||
Justin Bieber -> 0
|
||||
|
|
|
@ -9,7 +9,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -94,7 +94,7 @@ import java.io.IOException;
|
|||
* For more information on RabbitMQ high availability please see:
|
||||
* <a href="http://www.rabbitmq.com/ha.html">http://www.rabbitmq.com/ha.html</a>.
|
||||
*/
|
||||
public class RabbitMQFirehoseFactory implements FirehoseFactory
|
||||
public class RabbitMQFirehoseFactory implements FirehoseFactory<StringInputRowParser>
|
||||
{
|
||||
private static final Logger log = new Logger(RabbitMQFirehoseFactory.class);
|
||||
|
||||
|
@ -120,14 +120,18 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(StringInputRowParser firehoseParser) throws IOException
|
||||
{
|
||||
final StringInputRowParser stringParser = (StringInputRowParser) firehoseParser;
|
||||
|
||||
ConnectionOptions lyraOptions = new ConnectionOptions(this.connectionFactory);
|
||||
Config lyraConfig = new Config()
|
||||
.withRecoveryPolicy(new RetryPolicy()
|
||||
.withMaxRetries(config.getMaxRetries())
|
||||
.withRetryInterval(Duration.seconds(config.getRetryIntervalSeconds()))
|
||||
.withMaxDuration(Duration.seconds(config.getMaxDurationSeconds())));
|
||||
.withRecoveryPolicy(
|
||||
new RetryPolicy()
|
||||
.withMaxRetries(config.getMaxRetries())
|
||||
.withRetryInterval(Duration.seconds(config.getRetryIntervalSeconds()))
|
||||
.withMaxDuration(Duration.seconds(config.getMaxDurationSeconds()))
|
||||
);
|
||||
|
||||
String queue = config.getQueue();
|
||||
String exchange = config.getExchange();
|
||||
|
@ -139,26 +143,30 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory
|
|||
|
||||
final Connection connection = Connections.create(lyraOptions, lyraConfig);
|
||||
|
||||
connection.addShutdownListener(new ShutdownListener()
|
||||
{
|
||||
@Override
|
||||
public void shutdownCompleted(ShutdownSignalException cause)
|
||||
{
|
||||
log.warn(cause, "Connection closed!");
|
||||
}
|
||||
});
|
||||
connection.addShutdownListener(
|
||||
new ShutdownListener()
|
||||
{
|
||||
@Override
|
||||
public void shutdownCompleted(ShutdownSignalException cause)
|
||||
{
|
||||
log.warn(cause, "Connection closed!");
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
final Channel channel = connection.createChannel();
|
||||
channel.queueDeclare(queue, durable, exclusive, autoDelete, null);
|
||||
channel.queueBind(queue, exchange, routingKey);
|
||||
channel.addShutdownListener(new ShutdownListener()
|
||||
{
|
||||
@Override
|
||||
public void shutdownCompleted(ShutdownSignalException cause)
|
||||
{
|
||||
log.warn(cause, "Channel closed!");
|
||||
}
|
||||
});
|
||||
channel.addShutdownListener(
|
||||
new ShutdownListener()
|
||||
{
|
||||
@Override
|
||||
public void shutdownCompleted(ShutdownSignalException cause)
|
||||
{
|
||||
log.warn(cause, "Channel closed!");
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
// We create a QueueingConsumer that will not auto-acknowledge messages since that
|
||||
// happens on commit().
|
||||
|
@ -213,7 +221,7 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory
|
|||
return null;
|
||||
}
|
||||
|
||||
return parser.parse(new String(delivery.getBody()));
|
||||
return stringParser.parse(new String(delivery.getBody()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -27,7 +27,9 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.ByteBufferInputRowParser;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.impl.FileIteratingFirehose;
|
||||
|
@ -51,7 +53,7 @@ import java.util.zip.GZIPInputStream;
|
|||
/**
|
||||
* Builds firehoses that read from a predefined list of S3 objects and then dry up.
|
||||
*/
|
||||
public class StaticS3FirehoseFactory implements FirehoseFactory
|
||||
public class StaticS3FirehoseFactory implements FirehoseFactory<StringInputRowParser>
|
||||
{
|
||||
private static final Logger log = new Logger(StaticS3FirehoseFactory.class);
|
||||
|
||||
|
@ -88,7 +90,7 @@ public class StaticS3FirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(StringInputRowParser firehoseParser) throws IOException
|
||||
{
|
||||
Preconditions.checkNotNull(s3Client, "null s3Client");
|
||||
|
||||
|
@ -119,8 +121,9 @@ public class StaticS3FirehoseFactory implements FirehoseFactory
|
|||
|
||||
try {
|
||||
final InputStream innerInputStream = s3Client.getObject(
|
||||
new S3Bucket(s3Bucket), s3Object.getKey())
|
||||
.getDataInputStream();
|
||||
new S3Bucket(s3Bucket), s3Object.getKey()
|
||||
)
|
||||
.getDataInputStream();
|
||||
|
||||
final InputStream outerInputStream = s3Object.getKey().endsWith(".gz")
|
||||
? new GZIPInputStream(innerInputStream)
|
||||
|
@ -150,7 +153,7 @@ public class StaticS3FirehoseFactory implements FirehoseFactory
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
},
|
||||
parser
|
||||
(StringInputRowParser) firehoseParser
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.57-SNAPSHOT</version>
|
||||
<version>0.6.62-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -54,7 +54,7 @@ public class BrokerServerView implements TimelineServerView
|
|||
private final Map<String, ServerSelector> selectors;
|
||||
private final Map<String, VersionedIntervalTimeline<String, ServerSelector>> timelines;
|
||||
|
||||
private final QueryToolChestWarehouse warehose;
|
||||
private final QueryToolChestWarehouse warehouse;
|
||||
private final ObjectMapper smileMapper;
|
||||
private final HttpClient httpClient;
|
||||
private final ServerView baseView;
|
||||
|
@ -62,14 +62,14 @@ public class BrokerServerView implements TimelineServerView
|
|||
|
||||
@Inject
|
||||
public BrokerServerView(
|
||||
QueryToolChestWarehouse warehose,
|
||||
QueryToolChestWarehouse warehouse,
|
||||
ObjectMapper smileMapper,
|
||||
@Client HttpClient httpClient,
|
||||
ServerView baseView,
|
||||
ServerSelectorStrategy serverSelectorStrategy
|
||||
)
|
||||
{
|
||||
this.warehose = warehose;
|
||||
this.warehouse = warehouse;
|
||||
this.smileMapper = smileMapper;
|
||||
this.httpClient = httpClient;
|
||||
this.baseView = baseView;
|
||||
|
@ -149,7 +149,7 @@ public class BrokerServerView implements TimelineServerView
|
|||
|
||||
private DirectDruidClient makeDirectClient(DruidServer server)
|
||||
{
|
||||
return new DirectDruidClient(warehose, smileMapper, httpClient, server.getHost());
|
||||
return new DirectDruidClient(warehouse, smileMapper, httpClient, server.getHost());
|
||||
}
|
||||
|
||||
private QueryableDruidServer removeServer(DruidServer server)
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.metamx.common.logger.Logger;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -74,8 +75,7 @@ class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
|
|||
public byte[] put(ByteBuffer key, byte[] value)
|
||||
{
|
||||
numBytes += key.remaining() + value.length;
|
||||
byte[] retVal = super.put(key, value);
|
||||
return retVal;
|
||||
return super.put(key, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -98,4 +98,31 @@ class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
|
|||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] remove(Object key)
|
||||
{
|
||||
byte[] value = super.remove(key);
|
||||
if(value != null) {
|
||||
numBytes -= ((ByteBuffer)key).remaining() + value.length;
|
||||
}
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* We want keySet().iterator().remove() to account for object removal
|
||||
* The underlying Map calls this.remove(key) so we do not need to override this
|
||||
*/
|
||||
@Override
|
||||
public Set<ByteBuffer> keySet()
|
||||
{
|
||||
return super.keySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear()
|
||||
{
|
||||
numBytes = 0;
|
||||
super.clear();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -207,10 +207,13 @@ public class Announcer
|
|||
|
||||
Set<String> pathsToReinstate = Sets.newHashSet();
|
||||
for (String node : finalSubPaths.keySet()) {
|
||||
pathsToReinstate.add(ZKPaths.makePath(parentPath, node));
|
||||
String path = ZKPaths.makePath(parentPath, node);
|
||||
log.info("Node[%s] is added to reinstate.", path);
|
||||
pathsToReinstate.add(path);
|
||||
}
|
||||
|
||||
for (ChildData data : cache.getCurrentData()) {
|
||||
log.info("Node[%s] in cache is removed to reinstate.", data.getPath());
|
||||
pathsToReinstate.remove(data.getPath());
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.inject.Module;
|
|||
import com.google.inject.Provides;
|
||||
import com.google.inject.ProvisionException;
|
||||
import com.metamx.common.concurrent.ExecutorServiceConfig;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
|
@ -34,6 +35,7 @@ import io.druid.concurrent.Execs;
|
|||
import io.druid.guice.annotations.Global;
|
||||
import io.druid.guice.annotations.Processing;
|
||||
import io.druid.query.MetricsEmittingExecutorService;
|
||||
import io.druid.query.PrioritizedExecutorService;
|
||||
import io.druid.server.DruidProcessingConfig;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
|
@ -58,10 +60,17 @@ public class DruidProcessingModule implements Module
|
|||
@Provides
|
||||
@Processing
|
||||
@ManageLifecycle
|
||||
public ExecutorService getProcessingExecutorService(ExecutorServiceConfig config, ServiceEmitter emitter)
|
||||
public ExecutorService getProcessingExecutorService(
|
||||
ExecutorServiceConfig config,
|
||||
ServiceEmitter emitter,
|
||||
Lifecycle lifecycle
|
||||
)
|
||||
{
|
||||
return new MetricsEmittingExecutorService(
|
||||
Executors.newFixedThreadPool(config.getNumThreads(), Execs.makeThreadFactory(config.getFormatString())),
|
||||
PrioritizedExecutorService.create(
|
||||
lifecycle,
|
||||
config
|
||||
),
|
||||
emitter,
|
||||
new ServiceMetricEvent.Builder()
|
||||
);
|
||||
|
|
|
@ -30,6 +30,7 @@ import io.druid.guice.annotations.Self;
|
|||
import io.druid.initialization.DruidModule;
|
||||
import io.druid.server.DruidNode;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import io.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||
import io.druid.timeline.partition.LinearShardSpec;
|
||||
import io.druid.timeline.partition.NumberedShardSpec;
|
||||
import io.druid.timeline.partition.SingleDimensionShardSpec;
|
||||
|
@ -63,7 +64,8 @@ public class ServerModule implements DruidModule
|
|||
.registerSubtypes(
|
||||
new NamedType(SingleDimensionShardSpec.class, "single"),
|
||||
new NamedType(LinearShardSpec.class, "linear"),
|
||||
new NamedType(NumberedShardSpec.class, "numbered")
|
||||
new NamedType(NumberedShardSpec.class, "numbered"),
|
||||
new NamedType(HashBasedNumberedShardSpec.class, "hashed")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import io.druid.data.input.ByteBufferInputRowParser;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import io.druid.timeline.partition.ShardSpec;
|
||||
|
@ -32,7 +33,7 @@ import io.druid.timeline.partition.ShardSpec;
|
|||
public class DataSchema
|
||||
{
|
||||
private final String dataSource;
|
||||
private final ByteBufferInputRowParser parser;
|
||||
private final InputRowParser parser;
|
||||
private final AggregatorFactory[] aggregators;
|
||||
private final GranularitySpec granularitySpec;
|
||||
private final ShardSpec shardSpec;
|
||||
|
@ -40,14 +41,13 @@ public class DataSchema
|
|||
@JsonCreator
|
||||
public DataSchema(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("parser") ByteBufferInputRowParser parser,
|
||||
@JsonProperty("parser") InputRowParser parser,
|
||||
@JsonProperty("metricsSpec") AggregatorFactory[] aggregators,
|
||||
@JsonProperty("granularitySpec") GranularitySpec granularitySpec,
|
||||
@JsonProperty("shardSpec") ShardSpec shardSpec
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
Preconditions.checkNotNull(parser, "parser");
|
||||
Preconditions.checkNotNull(aggregators, "metrics");
|
||||
Preconditions.checkNotNull(granularitySpec, "granularitySpec");
|
||||
|
||||
|
@ -65,7 +65,7 @@ public class DataSchema
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
public ByteBufferInputRowParser getParser()
|
||||
public InputRowParser getParser()
|
||||
{
|
||||
return parser;
|
||||
}
|
||||
|
|
|
@ -185,8 +185,8 @@ public class RealtimeManager implements QuerySegmentWalker
|
|||
inputRow = firehose.nextRow();
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.info(e, "thrown away line due to exception");
|
||||
metrics.incrementThrownAway();
|
||||
log.debug(e, "thrown away line due to exception, considering unparseable");
|
||||
metrics.incrementUnparseable();
|
||||
continue;
|
||||
}
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.google.common.base.Predicate;
|
|||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -60,10 +61,10 @@ public class ClippedFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(InputRowParser parser) throws IOException
|
||||
{
|
||||
return new PredicateFirehose(
|
||||
delegate.connect(),
|
||||
delegate.connect(parser),
|
||||
new Predicate<InputRow>()
|
||||
{
|
||||
@Override
|
||||
|
@ -74,4 +75,10 @@ public class ClippedFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser getParser()
|
||||
{
|
||||
return delegate.getParser();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,13 +31,18 @@ import com.ircclouds.irc.api.domain.messages.ChannelPrivMsg;
|
|||
import com.ircclouds.irc.api.listeners.VariousMessageListenerAdapter;
|
||||
import com.ircclouds.irc.api.state.IIRCState;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.ByteBufferInputRowParser;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.ParseSpec;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
@ -94,14 +99,14 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|||
* );
|
||||
* }</pre>
|
||||
*/
|
||||
public class IrcFirehoseFactory implements FirehoseFactory
|
||||
public class IrcFirehoseFactory implements FirehoseFactory<IrcParser>
|
||||
{
|
||||
private static final Logger log = new Logger(IrcFirehoseFactory.class);
|
||||
|
||||
private final String nick;
|
||||
private final String host;
|
||||
private final List<String> channels;
|
||||
private final IrcDecoder decoder;
|
||||
private final IrcParser parser;
|
||||
|
||||
@JsonCreator
|
||||
public IrcFirehoseFactory(
|
||||
|
@ -114,11 +119,11 @@ public class IrcFirehoseFactory implements FirehoseFactory
|
|||
this.nick = nick;
|
||||
this.host = host;
|
||||
this.channels = channels;
|
||||
this.decoder = decoder;
|
||||
this.parser = new IrcParser(decoder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(final IrcParser firehoseParser) throws IOException
|
||||
{
|
||||
final IRCApi irc = new IRCApiImpl(false);
|
||||
final LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>> queue = new LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>>();
|
||||
|
@ -201,7 +206,7 @@ public class IrcFirehoseFactory implements FirehoseFactory
|
|||
while(true) {
|
||||
Pair<DateTime, ChannelPrivMsg> nextMsg = queue.take();
|
||||
try {
|
||||
nextRow = decoder.decodeMessage(nextMsg.lhs, nextMsg.rhs.getChannelName(), nextMsg.rhs.getText());
|
||||
nextRow = firehoseParser.parse(nextMsg);
|
||||
if(nextRow != null) return true;
|
||||
}
|
||||
catch (IllegalArgumentException iae) {
|
||||
|
@ -242,5 +247,11 @@ public class IrcFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public IrcParser getParser()
|
||||
{
|
||||
return parser;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.segment.realtime.firehose;
|
||||
|
||||
import com.ircclouds.irc.api.domain.messages.ChannelPrivMsg;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.ParseSpec;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IrcParser implements InputRowParser<Pair<DateTime, ChannelPrivMsg>>
|
||||
{
|
||||
private final IrcDecoder decoder;
|
||||
|
||||
public IrcParser(IrcDecoder decoder)
|
||||
{
|
||||
this.decoder = decoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRow parse(Pair<DateTime, ChannelPrivMsg> msg) throws FormattedException
|
||||
{
|
||||
return decoder.decodeMessage(msg.lhs, msg.rhs.getChannelName(), msg.rhs.getText());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ParseSpec getParseSpec()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser withParseSpec(ParseSpec parseSpec)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
|
@ -40,7 +40,7 @@ import java.util.LinkedList;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class LocalFirehoseFactory implements FirehoseFactory
|
||||
public class LocalFirehoseFactory implements FirehoseFactory<StringInputRowParser>
|
||||
{
|
||||
private final File baseDir;
|
||||
private final String filter;
|
||||
|
@ -77,7 +77,7 @@ public class LocalFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(StringInputRowParser firehoseParser) throws IOException
|
||||
{
|
||||
File[] foundFiles = baseDir.listFiles(
|
||||
new FilenameFilter()
|
||||
|
@ -125,7 +125,7 @@ public class LocalFirehoseFactory implements FirehoseFactory
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
},
|
||||
parser
|
||||
firehoseParser
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.metamx.emitter.EmittingLogger;
|
|||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -36,7 +37,7 @@ import java.util.concurrent.TimeUnit;
|
|||
/**
|
||||
* Creates firehoses that shut off at a particular time. Useful for limiting the lifespan of a realtime job.
|
||||
*/
|
||||
public class TimedShutoffFirehoseFactory implements FirehoseFactory
|
||||
public class TimedShutoffFirehoseFactory implements FirehoseFactory<InputRowParser>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(FirehoseFactory.class);
|
||||
private final FirehoseFactory delegateFactory;
|
||||
|
@ -53,9 +54,15 @@ public class TimedShutoffFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
public Firehose connect(InputRowParser parser) throws IOException
|
||||
{
|
||||
return new TimedShutoffFirehose();
|
||||
return new TimedShutoffFirehose(parser);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser getParser()
|
||||
{
|
||||
return delegateFactory.getParser();
|
||||
}
|
||||
|
||||
public class TimedShutoffFirehose implements Firehose
|
||||
|
@ -65,9 +72,9 @@ public class TimedShutoffFirehoseFactory implements FirehoseFactory
|
|||
private final Object shutdownLock = new Object();
|
||||
private volatile boolean shutdown = false;
|
||||
|
||||
public TimedShutoffFirehose() throws IOException
|
||||
public TimedShutoffFirehose(InputRowParser parser) throws IOException
|
||||
{
|
||||
firehose = delegateFactory.connect();
|
||||
firehose = delegateFactory.connect(parser);
|
||||
|
||||
exec = Executors.newScheduledThreadPool(
|
||||
1,
|
||||
|
@ -87,7 +94,8 @@ public class TimedShutoffFirehoseFactory implements FirehoseFactory
|
|||
shutdown = true;
|
||||
try {
|
||||
firehose.close();
|
||||
} catch (IOException e) {
|
||||
}
|
||||
catch (IOException e) {
|
||||
log.warn(e, "Failed to close delegate firehose, ignoring.");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.metamx.common.IAE;
|
|||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.SpatialDimensionSchema;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
|
@ -167,11 +168,17 @@ public class Sink implements Iterable<FireHydrant>
|
|||
|
||||
private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema)
|
||||
{
|
||||
List<SpatialDimensionSchema> spatialDimensionSchemas = schema.getParser() == null
|
||||
? Lists.<SpatialDimensionSchema>newArrayList()
|
||||
: schema.getParser()
|
||||
.getParseSpec()
|
||||
.getDimensionsSpec()
|
||||
.getSpatialDimensions();
|
||||
IncrementalIndex newIndex = new IncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMinTimestamp(minTimestamp)
|
||||
.withQueryGranularity(schema.getGranularitySpec().getQueryGranularity())
|
||||
.withSpatialDimensions(schema.getParser().getParseSpec().getDimensionsSpec().getSpatialDimensions())
|
||||
.withSpatialDimensions(spatialDimensionSchemas)
|
||||
.withMetrics(schema.getAggregators())
|
||||
.build()
|
||||
);
|
||||
|
|
|
@ -380,9 +380,9 @@ public class InfoResource
|
|||
}
|
||||
);
|
||||
Set<String> tiers = Sets.newHashSet(((LoadRule) rule).getTieredReplicants().keySet());
|
||||
tiers.remove(DruidServer.DEFAULT_TIER);
|
||||
String tier = DruidServer.DEFAULT_TIER;
|
||||
if (tiers.size() > 1) {
|
||||
tiers.remove(DruidServer.DEFAULT_TIER);
|
||||
if (!tiers.isEmpty()) {
|
||||
tier = tiers.iterator().next();
|
||||
}
|
||||
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue