mirror of https://github.com/apache/druid.git
Merge
This commit is contained in:
commit
1b20188ec7
Binary file not shown.
Binary file not shown.
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.72"
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.53-SNAPSHOT</version>
|
||||
<version>0.6.73-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.53-SNAPSHOT</version>
|
||||
<version>0.6.73-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.metamx.common.concurrent.ScheduledExecutors;
|
|||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.db.DbConnector;
|
||||
import io.druid.db.DbTablesConfig;
|
||||
import org.joda.time.Duration;
|
||||
import org.skife.jdbi.v2.Handle;
|
||||
|
@ -79,6 +80,12 @@ public class ConfigManager
|
|||
|
||||
this.selectStatement = String.format("SELECT payload FROM %s WHERE name = :name", configTable);
|
||||
this.insertStatement = String.format(
|
||||
DbConnector.isPostgreSQL(dbi) ?
|
||||
"BEGIN;\n" +
|
||||
"LOCK TABLE %1$s IN SHARE ROW EXCLUSIVE MODE;\n" +
|
||||
"WITH upsert AS (UPDATE %1$s SET payload=:payload WHERE name=:name RETURNING *)\n" +
|
||||
" INSERT INTO %1$s (name, payload) SELECT :name, :payload WHERE NOT EXISTS (SELECT * FROM upsert)\n;" +
|
||||
"COMMIT;" :
|
||||
"INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload",
|
||||
configTable
|
||||
);
|
||||
|
|
|
@ -22,11 +22,13 @@ package io.druid.concurrent;
|
|||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
import java.util.concurrent.ArrayBlockingQueue;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.RejectedExecutionHandler;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.SynchronousQueue;
|
||||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -60,19 +62,23 @@ public class Execs
|
|||
* @param capacity maximum capacity after which the executorService will block on accepting new tasks
|
||||
* @return ExecutorService which blocks accepting new tasks when the capacity reached
|
||||
*/
|
||||
public static ExecutorService newBlockingSingleThreaded(String nameFormat, int capacity)
|
||||
public static ExecutorService newBlockingSingleThreaded(final String nameFormat, final int capacity)
|
||||
{
|
||||
final BlockingQueue<Runnable> queue;
|
||||
if (capacity > 0) {
|
||||
queue = new ArrayBlockingQueue<>(capacity);
|
||||
} else {
|
||||
queue = new SynchronousQueue<>();
|
||||
}
|
||||
return new ThreadPoolExecutor(
|
||||
1, 1,
|
||||
0L, TimeUnit.MILLISECONDS,
|
||||
new ArrayBlockingQueue<Runnable>(capacity), makeThreadFactory(nameFormat)
|
||||
, new RejectedExecutionHandler()
|
||||
1, 1, 0L, TimeUnit.MILLISECONDS, queue, makeThreadFactory(nameFormat),
|
||||
new RejectedExecutionHandler()
|
||||
{
|
||||
@Override
|
||||
public void rejectedExecution(Runnable r, ThreadPoolExecutor executor)
|
||||
{
|
||||
try {
|
||||
((ArrayBlockingQueue) executor.getQueue()).put(r);
|
||||
executor.getQueue().put(r);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RejectedExecutionException("Got Interrupted while adding to the Queue");
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.skife.jdbi.v2.IDBI;
|
|||
import org.skife.jdbi.v2.tweak.HandleCallback;
|
||||
|
||||
import javax.sql.DataSource;
|
||||
import java.sql.SQLException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -44,6 +45,10 @@ public class DbConnector
|
|||
dbi,
|
||||
segmentTableName,
|
||||
String.format(
|
||||
isPostgreSQL(dbi) ?
|
||||
"CREATE TABLE %1$s (id VARCHAR(255) NOT NULL, dataSource VARCHAR(255) NOT NULL, created_date TEXT NOT NULL, start TEXT NOT NULL, \"end\" TEXT NOT NULL, partitioned SMALLINT NOT NULL, version TEXT NOT NULL, used BOOLEAN NOT NULL, payload bytea NOT NULL, PRIMARY KEY (id));" +
|
||||
"CREATE INDEX ON %1$s(dataSource);"+
|
||||
"CREATE INDEX ON %1$s(used);":
|
||||
"CREATE table %s (id VARCHAR(255) NOT NULL, dataSource VARCHAR(255) NOT NULL, created_date TINYTEXT NOT NULL, start TINYTEXT NOT NULL, end TINYTEXT NOT NULL, partitioned BOOLEAN NOT NULL, version TINYTEXT NOT NULL, used BOOLEAN NOT NULL, payload LONGTEXT NOT NULL, INDEX(dataSource), INDEX(used), PRIMARY KEY (id))",
|
||||
segmentTableName
|
||||
)
|
||||
|
@ -56,6 +61,9 @@ public class DbConnector
|
|||
dbi,
|
||||
ruleTableName,
|
||||
String.format(
|
||||
isPostgreSQL(dbi) ?
|
||||
"CREATE TABLE %1$s (id VARCHAR(255) NOT NULL, dataSource VARCHAR(255) NOT NULL, version TEXT NOT NULL, payload bytea NOT NULL, PRIMARY KEY (id));"+
|
||||
"CREATE INDEX ON %1$s(dataSource);":
|
||||
"CREATE table %s (id VARCHAR(255) NOT NULL, dataSource VARCHAR(255) NOT NULL, version TINYTEXT NOT NULL, payload LONGTEXT NOT NULL, INDEX(dataSource), PRIMARY KEY (id))",
|
||||
ruleTableName
|
||||
)
|
||||
|
@ -68,6 +76,8 @@ public class DbConnector
|
|||
dbi,
|
||||
configTableName,
|
||||
String.format(
|
||||
isPostgreSQL(dbi) ?
|
||||
"CREATE TABLE %s (name VARCHAR(255) NOT NULL, payload bytea NOT NULL, PRIMARY KEY(name))":
|
||||
"CREATE table %s (name VARCHAR(255) NOT NULL, payload BLOB NOT NULL, PRIMARY KEY(name))",
|
||||
configTableName
|
||||
)
|
||||
|
@ -80,6 +90,17 @@ public class DbConnector
|
|||
dbi,
|
||||
taskTableName,
|
||||
String.format(
|
||||
isPostgreSQL(dbi) ?
|
||||
"CREATE TABLE %1$s (\n"
|
||||
+ " id varchar(255) NOT NULL,\n"
|
||||
+ " created_date TEXT NOT NULL,\n"
|
||||
+ " datasource varchar(255) NOT NULL,\n"
|
||||
+ " payload bytea NOT NULL,\n"
|
||||
+ " status_payload bytea NOT NULL,\n"
|
||||
+ " active SMALLINT NOT NULL DEFAULT '0',\n"
|
||||
+ " PRIMARY KEY (id)\n"
|
||||
+ ");\n" +
|
||||
"CREATE INDEX ON %1$s(active, created_date);":
|
||||
"CREATE TABLE `%s` (\n"
|
||||
+ " `id` varchar(255) NOT NULL,\n"
|
||||
+ " `created_date` tinytext NOT NULL,\n"
|
||||
|
@ -87,7 +108,8 @@ public class DbConnector
|
|||
+ " `payload` longblob NOT NULL,\n"
|
||||
+ " `status_payload` longblob NOT NULL,\n"
|
||||
+ " `active` tinyint(1) NOT NULL DEFAULT '0',\n"
|
||||
+ " PRIMARY KEY (`id`)\n"
|
||||
+ " PRIMARY KEY (`id`),\n"
|
||||
+ " KEY (active, created_date(100))\n"
|
||||
+ ")",
|
||||
taskTableName
|
||||
)
|
||||
|
@ -100,6 +122,14 @@ public class DbConnector
|
|||
dbi,
|
||||
taskLogsTableName,
|
||||
String.format(
|
||||
isPostgreSQL(dbi) ?
|
||||
"CREATE TABLE %1$s (\n"
|
||||
+ " id bigserial NOT NULL,\n"
|
||||
+ " task_id varchar(255) DEFAULT NULL,\n"
|
||||
+ " log_payload bytea,\n"
|
||||
+ " PRIMARY KEY (id)\n"
|
||||
+ ");\n"+
|
||||
"CREATE INDEX ON %1$s(task_id);":
|
||||
"CREATE TABLE `%s` (\n"
|
||||
+ " `id` bigint(20) NOT NULL AUTO_INCREMENT,\n"
|
||||
+ " `task_id` varchar(255) DEFAULT NULL,\n"
|
||||
|
@ -118,6 +148,14 @@ public class DbConnector
|
|||
dbi,
|
||||
taskLocksTableName,
|
||||
String.format(
|
||||
isPostgreSQL(dbi) ?
|
||||
"CREATE TABLE %1$s (\n"
|
||||
+ " id bigserial NOT NULL,\n"
|
||||
+ " task_id varchar(255) DEFAULT NULL,\n"
|
||||
+ " lock_payload bytea,\n"
|
||||
+ " PRIMARY KEY (id)\n"
|
||||
+ ");\n"+
|
||||
"CREATE INDEX ON %1$s(task_id);":
|
||||
"CREATE TABLE `%s` (\n"
|
||||
+ " `id` bigint(20) NOT NULL AUTO_INCREMENT,\n"
|
||||
+ " `task_id` varchar(255) DEFAULT NULL,\n"
|
||||
|
@ -143,8 +181,12 @@ public class DbConnector
|
|||
@Override
|
||||
public Void withHandle(Handle handle) throws Exception
|
||||
{
|
||||
if ( !handle.getConnection().getMetaData().getDatabaseProductName().contains("PostgreSQL") ) {
|
||||
List<Map<String, Object>> table = handle.select(String.format("SHOW tables LIKE '%s'", tableName));
|
||||
List<Map<String, Object>> table;
|
||||
if ( isPostgreSQL(dbi) ) {
|
||||
table = handle.select(String.format("SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = 'public' AND tablename LIKE '%s'", tableName));
|
||||
} else {
|
||||
table = handle.select(String.format("SHOW tables LIKE '%s'", tableName));
|
||||
}
|
||||
|
||||
if (table.isEmpty()) {
|
||||
log.info("Creating table[%s]", tableName);
|
||||
|
@ -152,7 +194,7 @@ public class DbConnector
|
|||
} else {
|
||||
log.info("Table[%s] existed: [%s]", tableName, table);
|
||||
}
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -163,6 +205,25 @@ public class DbConnector
|
|||
}
|
||||
}
|
||||
|
||||
public static Boolean isPostgreSQL(final IDBI dbi)
|
||||
{
|
||||
return dbi.withHandle(
|
||||
new HandleCallback<Boolean>()
|
||||
{
|
||||
@Override
|
||||
public Boolean withHandle(Handle handle) throws Exception
|
||||
{
|
||||
return isPostgreSQL(handle);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
public static Boolean isPostgreSQL(final Handle handle) throws SQLException
|
||||
{
|
||||
return handle.getConnection().getMetaData().getDatabaseProductName().contains("PostgreSQL");
|
||||
}
|
||||
|
||||
private final Supplier<DbConnectorConfig> config;
|
||||
private final Supplier<DbTablesConfig> dbTables;
|
||||
private final DBI dbi;
|
||||
|
|
|
@ -20,6 +20,8 @@
|
|||
package io.druid.concurrent;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -30,23 +32,46 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
public class ExecsTest
|
||||
{
|
||||
private static final Logger log = new Logger(ExecsTest.class);
|
||||
|
||||
@Test
|
||||
public void testBlockingExecutorService() throws Exception
|
||||
public void testBlockingExecutorServiceZeroCapacity() throws Exception
|
||||
{
|
||||
final int capacity = 3;
|
||||
final ExecutorService blockingExecutor = Execs.newBlockingSingleThreaded("test%d", capacity);
|
||||
final CountDownLatch queueFullSignal = new CountDownLatch(capacity + 1);
|
||||
final CountDownLatch taskCompletedSignal = new CountDownLatch(2 * capacity);
|
||||
runTest(0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBlockingExecutorServiceOneCapacity() throws Exception
|
||||
{
|
||||
runTest(1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBlockingExecutorServiceThreeCapacity() throws Exception
|
||||
{
|
||||
runTest(3);
|
||||
}
|
||||
|
||||
private static void runTest(final int capacity) throws Exception
|
||||
{
|
||||
final int nTasks = (capacity + 1) * 3;
|
||||
final ExecutorService blockingExecutor = Execs.newBlockingSingleThreaded("ExecsTest-Blocking-%d", capacity);
|
||||
final CountDownLatch queueShouldBeFullSignal = new CountDownLatch(capacity + 1);
|
||||
final CountDownLatch taskCompletedSignal = new CountDownLatch(nTasks);
|
||||
final CountDownLatch taskStartSignal = new CountDownLatch(1);
|
||||
final AtomicInteger producedCount = new AtomicInteger();
|
||||
final AtomicInteger consumedCount = new AtomicInteger();
|
||||
ExecutorService producer = Executors.newSingleThreadExecutor();
|
||||
final ExecutorService producer = Executors.newSingleThreadExecutor(
|
||||
new ThreadFactoryBuilder().setNameFormat(
|
||||
"ExecsTest-Producer-%d"
|
||||
).build()
|
||||
);
|
||||
producer.submit(
|
||||
new Runnable()
|
||||
{
|
||||
public void run()
|
||||
{
|
||||
for (int i = 0; i < 2 * capacity; i++) {
|
||||
for (int i = 0; i < nTasks; i++) {
|
||||
final int taskID = i;
|
||||
System.out.println("Produced task" + taskID);
|
||||
blockingExecutor.submit(
|
||||
|
@ -55,7 +80,7 @@ public class ExecsTest
|
|||
@Override
|
||||
public void run()
|
||||
{
|
||||
System.out.println("Starting task" + taskID);
|
||||
log.info("Starting task: %s", taskID);
|
||||
try {
|
||||
taskStartSignal.await();
|
||||
consumedCount.incrementAndGet();
|
||||
|
@ -64,29 +89,31 @@ public class ExecsTest
|
|||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
System.out.println("Completed task" + taskID);
|
||||
log.info("Completed task: %s", taskID);
|
||||
}
|
||||
}
|
||||
);
|
||||
producedCount.incrementAndGet();
|
||||
queueFullSignal.countDown();
|
||||
queueShouldBeFullSignal.countDown();
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
queueFullSignal.await();
|
||||
// verify that the producer blocks
|
||||
queueShouldBeFullSignal.await();
|
||||
// Verify that the producer blocks. I don't think it's possible to be sure that the producer is blocking (since
|
||||
// it could be doing nothing for any reason). But waiting a short period of time and checking that it hasn't done
|
||||
// anything should hopefully be sufficient.
|
||||
Thread.sleep(500);
|
||||
Assert.assertEquals(capacity + 1, producedCount.get());
|
||||
// let the tasks run
|
||||
taskStartSignal.countDown();
|
||||
// wait until all tasks complete
|
||||
taskCompletedSignal.await();
|
||||
// verify all tasks consumed
|
||||
Assert.assertEquals(2 * capacity, consumedCount.get());
|
||||
Assert.assertEquals(nTasks, consumedCount.get());
|
||||
// cleanup
|
||||
blockingExecutor.shutdown();
|
||||
producer.shutdown();
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Experimental features are features we have developed but have not fully tested in a production environment. If you choose to try them out, there will likely to edge cases that we have not covered. We would love feedback on any of these features, whether they are bug reports, suggestions for improvement, or letting us know they work as intended.
|
||||
# About Experimental Features
|
||||
Experimental features are features we have developed but have not fully tested in a production environment. If you choose to try them out, there will likely be edge cases that we have not covered. We would love feedback on any of these features, whether they are bug reports, suggestions for improvement, or letting us know they work as intended.
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Aggregations
|
||||
Aggregations are specifications of processing over metrics available in Druid.
|
||||
Available aggregations are:
|
||||
|
||||
|
@ -81,3 +82,13 @@ All JavaScript functions must return numerical values.
|
|||
"fnReset" : "function() { return 10; }"
|
||||
}
|
||||
```
|
||||
|
||||
### Complex aggregators
|
||||
|
||||
#### `hyperUnique` aggregator
|
||||
|
||||
`hyperUnique` uses [Hyperloglog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to compute the estimated cardinality of a dimension.
|
||||
|
||||
```json
|
||||
{ "type" : "hyperUnique", "name" : <output_name>, "fieldName" : <metric_name> }
|
||||
```
|
||||
|
|
|
@ -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": "hashed"
|
||||
"targetPartitionSize": 5000000
|
||||
},
|
||||
"updaterJobSpec": {
|
||||
|
@ -145,12 +146,22 @@ 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 hashed.
|
||||
|
||||
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 hashed partition type, the number of partitions is determined based on the targetPartitionSize and cardinality of input set and the data is partitioned based on the hashcode of the row.
|
||||
|
||||
It is recommended to use Hashed partition as it is more efficient than singleDimension since it does not need to determine the dimension for creating partitions.
|
||||
Hashing also gives better distribution of data resulting in equal sized partitons and improving query performance
|
||||
|
||||
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|
|
||||
|
|
|
@ -67,7 +67,7 @@ You can then use the EC2 dashboard to locate the instance and confirm that it ha
|
|||
If both the instance and the Druid cluster launch successfully, a few minutes later other messages to STDOUT should follow with information returned from EC2, including the instance ID:
|
||||
|
||||
Started cluster of 1 instances
|
||||
Cluster{instances=[Instance{roles=[zookeeper, druid-mysql, druid-master, druid-broker, druid-compute, druid-realtime], publicIp= ...
|
||||
Cluster{instances=[Instance{roles=[zookeeper, druid-mysql, druid-coordinator, druid-broker, druid-historical, druid-realtime], publicIp= ...
|
||||
|
||||
The final message will contain login information for the instance.
|
||||
|
||||
|
|
|
@ -0,0 +1,121 @@
|
|||
---
|
||||
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
|
||||
|
||||
# Change these to make Druid faster
|
||||
druid.processing.buffer.sizeBytes=100000000
|
||||
druid.processing.numThreads=1
|
||||
|
||||
```
|
||||
|
||||
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 historical 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.|10485760 (10MB)|
|
||||
|`druid.broker.cache.initialSize`|The initial size of the cache in bytes.|500000|
|
||||
|`druid.broker.cache.logEvictionCount`|If this is non-zero, there will be an eviction of entries.|0|
|
||||
|
||||
#### 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
|
||||
------------------
|
||||
|
||||
|
|
|
@ -14,7 +14,7 @@ As a special case, the absolute minimum setup is one of the standalone examples
|
|||
Minimum Physical Layout: Experimental Testing with 4GB of RAM
|
||||
-------------------------------------------------------------
|
||||
|
||||
This layout can be used to load some data from deep storage onto a Druid compute node for the first time. A minimal physical layout for a 1 or 2 core machine with 4GB of RAM is:
|
||||
This layout can be used to load some data from deep storage onto a Druid historical node for the first time. A minimal physical layout for a 1 or 2 core machine with 4GB of RAM is:
|
||||
|
||||
1. node1: [Coordinator](Coordinator.html) + metadata service + zookeeper + [Historical](Historical.html)
|
||||
2. transient nodes: [Indexing Service](Indexing-Service.html)
|
||||
|
@ -37,7 +37,7 @@ A minimal physical layout not constrained by cores that demonstrates parallel qu
|
|||
7. node7: [Realtime](Realtime.html) (m1.small or m1.medium or m1.large)
|
||||
8. transient nodes: [Indexing Service](Indexing-Service.html)
|
||||
|
||||
This layout naturally lends itself to adding more RAM and core to Compute nodes, and to adding many more Compute nodes. Depending on the actual load, the Master, metadata server, and Zookeeper might need to use larger machines.
|
||||
This layout naturally lends itself to adding more RAM and core to Historical nodes, and to adding many more Historical nodes. Depending on the actual load, the Coordinator, metadata server, and Zookeeper might need to use larger machines.
|
||||
|
||||
High Availability Physical Layout
|
||||
---------------------------------
|
||||
|
@ -63,14 +63,14 @@ An HA layout allows full rolling restarts and heavy volume:
|
|||
Sizing for Cores and RAM
|
||||
------------------------
|
||||
|
||||
The Compute and Broker nodes will use as many cores as are available, depending on usage, so it is best to keep these on dedicated machines. The upper limit of effectively utilized cores is not well characterized yet and would depend on types of queries, query load, and the schema. Compute daemons should have a heap a size of at least 1GB per core for normal usage, but could be squeezed into a smaller heap for testing. Since in-memory caching is essential for good performance, even more RAM is better. Broker nodes will use RAM for caching, so they do more than just route queries.
|
||||
The Historical and Broker nodes will use as many cores as are available, depending on usage, so it is best to keep these on dedicated machines. The upper limit of effectively utilized cores is not well characterized yet and would depend on types of queries, query load, and the schema. Historical daemons should have a heap a size of at least 1GB per core for normal usage, but could be squeezed into a smaller heap for testing. Since in-memory caching is essential for good performance, even more RAM is better. Broker nodes will use RAM for caching, so they do more than just route queries.
|
||||
|
||||
The effective utilization of cores by Zookeeper, MySQL, and Master nodes is likely to be between 1 and 2 for each process/daemon, so these could potentially share a machine with lots of cores. These daemons work with heap a size between 500MB and 1GB.
|
||||
The effective utilization of cores by Zookeeper, MySQL, and Coordinator nodes is likely to be between 1 and 2 for each process/daemon, so these could potentially share a machine with lots of cores. These daemons work with heap a size between 500MB and 1GB.
|
||||
|
||||
Storage
|
||||
-------
|
||||
|
||||
Indexed segments should be kept in a permanent store accessible by all nodes like AWS S3 or HDFS or equivalent. Currently Druid supports S3, but this will be extended soon.
|
||||
Indexed segments should be kept in a permanent store accessible by all nodes like AWS S3 or HDFS or equivalent. Refer to [Deep-Storage](deep-storage.html) for more details on supported storage types.
|
||||
|
||||
Local disk ("ephemeral" on AWS EC2) for caching is recommended over network mounted storage (example of mounted: AWS EBS, Elastic Block Store) in order to avoid network delays during times of heavy usage. If your data center is suitably provisioned for networked storage, perhaps with separate LAN/NICs just for storage, then mounted might work fine.
|
||||
|
||||
|
@ -85,7 +85,7 @@ druid.host=someHostOrIPaddrWithPort
|
|||
druid.port=8080
|
||||
```
|
||||
|
||||
`druid.server.type` should be set to "historical" for your compute nodes and realtime for the realtime nodes. The master will only assign segments to a "historical" node and the broker has some intelligence around its ability to cache results when talking to a realtime node. This does not need to be set for the master or the broker.
|
||||
`druid.server.type` should be set to "historical" for your historical nodes and realtime for the realtime nodes. The Coordinator will only assign segments to a "historical" node and the broker has some intelligence around its ability to cache results when talking to a realtime node. This does not need to be set for the coordinator or the broker.
|
||||
|
||||
`druid.host` should be set to the hostname and port that can be used to talk to the given server process. Basically, someone should be able to send a request to http://${druid.host}/ and actually talk to the process.
|
||||
|
||||
|
|
|
@ -193,7 +193,7 @@ This module is required by nodes that can serve queries.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.query.chunkPeriod`|Long interval queries may be broken into shorter interval queries.|P1M|
|
||||
|`druid.query.chunkPeriod`|Long interval queries may be broken into shorter interval queries.|0|
|
||||
|
||||
#### GroupBy Query Config
|
||||
|
||||
|
|
|
@ -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
|
||||
|
||||
```
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
A data source is the Druid equivalent of a database table. However, a query can also masquerade as a data source, providing subquery-like functionality. Query data sources are currently only supported by [GroupBy](GroupByQuery.html) queries.
|
||||
|
||||
### Table Data Source
|
||||
The table data source the most common type. It's represented by a string, or by the full structure:
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "table",
|
||||
"name": <string_value>
|
||||
}
|
||||
```
|
||||
|
||||
### Query Data Source
|
||||
```json
|
||||
{
|
||||
"type": "query",
|
||||
"query": {
|
||||
"type": "groupBy",
|
||||
...
|
||||
}
|
||||
}
|
||||
```
|
|
@ -0,0 +1,87 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Data Formats for Ingestion
|
||||
==========================
|
||||
|
||||
Druid can ingest data in JSON, CSV, or TSV. While most examples in the documentation use data in JSON format, it is not difficult to configure Druid to ingest CSV or TSV data.
|
||||
|
||||
## Formatting the Data
|
||||
The following are three samples of the data used in the [Wikipedia example](Tutorial:-Loading-Your-Data-Part-1.html).
|
||||
|
||||
_JSON_
|
||||
|
||||
```json
|
||||
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
|
||||
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
||||
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "cancer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
||||
```
|
||||
|
||||
_CSV_
|
||||
|
||||
```
|
||||
2013-08-31T01:02:33Z,"Gypsy Danger","en","nuclear","true","true","false","false","article","North America","United States","Bay Area","San Francisco",57,200,-143
|
||||
2013-08-31T03:32:45Z,"Striker Eureka","en","speed","false","true","true","false","wikipedia","Australia","Australia","Cantebury","Syndey",459,129,330
|
||||
2013-08-31T07:11:21Z,"Cherno Alpha","ru","masterYi","false","true","true","false","article","Asia","Russia","Oblast","Moscow",123,12,111
|
||||
2013-08-31T11:58:39Z,"Crimson Typhoon","zh","triplets","true","false","true","false","wikipedia","Asia","China","Shanxi","Taiyuan",905,5,900
|
||||
2013-08-31T12:41:27Z,"Coyote Tango","ja","cancer","true","false","true","false","wikipedia","Asia","Japan","Kanto","Tokyo",1,10,-9
|
||||
```
|
||||
|
||||
_TSV_
|
||||
|
||||
```
|
||||
2013-08-31T01:02:33Z "Gypsy Danger" "en" "nuclear" "true" "true" "false" "false" "article" "North America" "United States" "Bay Area" "San Francisco" 57 200 -143
|
||||
2013-08-31T03:32:45Z "Striker Eureka" "en" "speed" "false" "true" "true" "false" "wikipedia" "Australia" "Australia" "Cantebury" "Syndey" 459 129 330
|
||||
2013-08-31T07:11:21Z "Cherno Alpha" "ru" "masterYi" "false" "true" "true" "false" "article" "Asia" "Russia" "Oblast" "Moscow" 123 12 111
|
||||
2013-08-31T11:58:39Z "Crimson Typhoon" "zh" "triplets" "true" "false" "true" "false" "wikipedia" "Asia" "China" "Shanxi" "Taiyuan" 905 5 900
|
||||
2013-08-31T12:41:27Z "Coyote Tango" "ja" "cancer" "true" "false" "true" "false" "wikipedia" "Asia" "Japan" "Kanto" "Tokyo" 1 10 -9
|
||||
```
|
||||
|
||||
Note that the CSV and TSV data do not contain column heads. This becomes important when you specify the data for ingesting.
|
||||
|
||||
## Configuring Ingestion For the Indexing Service
|
||||
If you use the [indexing service](Indexing-Service.html) for ingesting the data, a [task](Tasks.html) must be configured and submitted. Tasks are configured with a JSON object which, among other things, specifies the data source and type. In the Wikipedia example, JSON data was read from a local file. The task spec contains a firehose element to specify this:
|
||||
|
||||
```json
|
||||
"firehose" : {
|
||||
"type" : "local",
|
||||
"baseDir" : "examples/indexing",
|
||||
"filter" : "wikipedia_data.json",
|
||||
"parser" : {
|
||||
"timestampSpec" : {
|
||||
"column" : "timestamp"
|
||||
},
|
||||
"data" : {
|
||||
"format" : "json",
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
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 cannot contain the column names (no header is allowed), these must be added before that data can be processed:
|
||||
|
||||
```json
|
||||
"firehose" : {
|
||||
"type" : "local",
|
||||
"baseDir" : "examples/indexing/",
|
||||
"filter" : "wikipedia_data.csv",
|
||||
"parser" : {
|
||||
"timestampSpec" : {
|
||||
"column" : "timestamp"
|
||||
},
|
||||
"data" : {
|
||||
"type" : "csv",
|
||||
"columns" : ["timestamp","page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"],
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
Note also that the filename extension and the data type were changed to "csv". For the TSV data, the same changes are made but with "tsv" for the filename extension and the data type.
|
||||
|
|
@ -1,9 +1,10 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Deep Storage
|
||||
Deep storage is where segments are stored. It is a storage mechanism that Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid nodes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented.
|
||||
|
||||
The currently supported types of deep storage follow.
|
||||
The currently supported types of deep storage follow. Other deep-storage options, such as [Cassandra](http://planetcassandra.org/blog/post/cassandra-as-a-deep-storage-mechanism-for-druid-real-time-analytics-engine/), have been developed by members of the community.
|
||||
|
||||
## S3-compatible
|
||||
|
||||
|
|
|
@ -1,6 +1,8 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Transforming Dimension Values
|
||||
The following JSON fields can be used in a query to operate on dimension values.
|
||||
|
||||
## DimensionSpec
|
||||
|
||||
|
@ -8,7 +10,7 @@ layout: doc_page
|
|||
|
||||
### DefaultDimensionSpec
|
||||
|
||||
Returns dimension values as is and optionally renames renames the dimension.
|
||||
Returns dimension values as is and optionally renames the dimension.
|
||||
|
||||
```json
|
||||
{ "type" : "default", "dimension" : <dimension>, "outputName": <output_name> }
|
||||
|
|
|
@ -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.72
|
||||
./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.72-bin.tar.gz) a stand-alone tarball and run it:
|
||||
|
||||
``` bash
|
||||
tar -xzf druid-services-0.X.X-bin.tar.gz
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
#Query Filters
|
||||
A filter is a JSON object indicating which rows of data should be included in the computation for a query. It’s essentially the equivalent of the WHERE clause in SQL. Druid supports the following types of filters.
|
||||
|
||||
### Selector filter
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Geographic Queries
|
||||
Druid supports filtering specially spatially indexed columns based on an origin and a bound.
|
||||
|
||||
# Spatial Indexing
|
||||
|
|
|
@ -1,13 +1,14 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
The granularity field determines how data gets bucketed across the time dimension, i.e how it gets aggregated by hour, day, minute, etc.
|
||||
# Aggregation Granularity
|
||||
The granularity field determines how data gets bucketed across the time dimension, or how it gets aggregated by hour, day, minute, etc.
|
||||
|
||||
It can be specified either as a string for simple granularities or as an object for arbitrary granularities.
|
||||
|
||||
### Simple Granularities
|
||||
|
||||
Simple granularities are specified as a string and bucket timestamps by their UTC time (i.e. days start at 00:00 UTC).
|
||||
Simple granularities are specified as a string and bucket timestamps by their UTC time (e.g., days start at 00:00 UTC).
|
||||
|
||||
Supported granularity strings are: `all`, `none`, `minute`, `fifteen_minute`, `thirty_minute`, `hour` and `day`
|
||||
|
||||
|
@ -34,25 +35,21 @@ This chunks up every hour on the half-hour.
|
|||
|
||||
### Period Granularities
|
||||
|
||||
Period granularities are specified as arbitrary period combinations of years, months, weeks, hours, minutes and seconds (e.g. P2W, P3M, PT1H30M, PT0.750S) in ISO8601 format.
|
||||
Period granularities are specified as arbitrary period combinations of years, months, weeks, hours, minutes and seconds (e.g. P2W, P3M, PT1H30M, PT0.750S) in ISO8601 format. They support specifying a time zone which determines where period boundaries start as well as the timezone of the returned timestamps. By default, years start on the first of January, months start on the first of the month and weeks start on Mondays unless an origin is specified.
|
||||
|
||||
They support specifying a time zone which determines where period boundaries start and also determines the timezone of the returned timestamps.
|
||||
|
||||
By default years start on the first of January, months start on the first of the month and weeks start on Mondays unless an origin is specified.
|
||||
|
||||
Time zone is optional (defaults to UTC)
|
||||
Origin is optional (defaults to 1970-01-01T00:00:00 in the given time zone)
|
||||
Time zone is optional (defaults to UTC). Origin is optional (defaults to 1970-01-01T00:00:00 in the given time zone).
|
||||
|
||||
```
|
||||
{"type": "period", "period": "P2D", "timeZone": "America/Los_Angeles"}
|
||||
```
|
||||
|
||||
This will bucket by two day chunks in the Pacific timezone.
|
||||
This will bucket by two-day chunks in the Pacific timezone.
|
||||
|
||||
```
|
||||
{"type": "period", "period": "P3M", "timeZone": "America/Los_Angeles", "origin": "2012-02-01T00:00:00-08:00"}
|
||||
```
|
||||
|
||||
This will bucket by 3 month chunks in the Pacific timezone where the three-month quarters are defined as starting from February.
|
||||
This will bucket by 3-month chunks in the Pacific timezone where the three-month quarters are defined as starting from February.
|
||||
|
||||
Supported time zones: timezone support is provided by the [Joda Time library](http://www.joda.org), which uses the standard IANA time zones. [Joda Time supported timezones](http://joda-time.sourceforge.net/timezones.html)
|
||||
#### Supported Time Zones
|
||||
Timezone support is provided by the [Joda Time library](http://www.joda.org), which uses the standard IANA time zones. See the [Joda Time supported timezones](http://joda-time.sourceforge.net/timezones.html).
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# groupBy Queries
|
||||
These types of queries take a groupBy query object and return an array of JSON objects where each object represents a grouping asked for by the query. Note: If you only want to do straight aggregates for some time range, we highly recommend using [TimeseriesQueries](TimeseriesQuery.html) instead. The performance will be substantially better.
|
||||
An example groupBy query object is shown below:
|
||||
|
||||
|
@ -47,7 +48,7 @@ There are 9 main parts to a groupBy query:
|
|||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|queryType|This String should always be "groupBy"; this is the first thing Druid looks at to figure out how to interpret the query|yes|
|
||||
|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes|
|
||||
|dataSource|A String defining the data source to query, very similar to a table in a relational database, or a [DataSource](DataSource.html) structure.|yes|
|
||||
|dimensions|A JSON list of dimensions to do the groupBy over|yes|
|
||||
|orderBy|See [OrderBy](OrderBy.html).|no|
|
||||
|having|See [Having](Having.html).|no|
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Filter groupBy Query Results
|
||||
A having clause is a JSON object identifying which rows from a groupBy query should be returned, by specifying conditions on aggregated values.
|
||||
|
||||
It is essentially the equivalent of the HAVING clause in SQL.
|
||||
|
|
|
@ -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/historical/_default
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:#{DRUID_VERSION}"]
|
||||
|
||||
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.72"]
|
||||
|
||||
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.72","io.druid.extensions:druid-kafka-seven:0.6.72"]
|
||||
|
||||
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,6 +1,10 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
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.
|
||||
|
@ -20,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:
|
||||
|
@ -109,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).
|
||||
|
||||
|
|
|
@ -22,7 +22,7 @@ druid.storage.baseKey=sample
|
|||
```
|
||||
|
||||
## I don't see my Druid segments on my historical nodes
|
||||
You can check the coordinator console located at <COORDINATOR_IP>:<PORT>/cluster.html. Make sure that your segments have actually loaded on [historical nodes](Historical.html). If your segments are not present, check the coordinator logs for messages about capacity of replication errors. One reason that segments are not downloaded is because historical nodes have maxSizes that are too small, making them incapable of downloading more data. You can change that with (for example):
|
||||
You can check the coordinator console located at `<COORDINATOR_IP>:<PORT>/cluster.html`. Make sure that your segments have actually loaded on [historical nodes](Historical.html). If your segments are not present, check the coordinator logs for messages about capacity of replication errors. One reason that segments are not downloaded is because historical nodes have maxSizes that are too small, making them incapable of downloading more data. You can change that with (for example):
|
||||
|
||||
```
|
||||
-Ddruid.segmentCache.locations=[{"path":"/tmp/druid/storageLocation","maxSize":"500000000000"}]
|
||||
|
@ -31,7 +31,7 @@ You can check the coordinator console located at <COORDINATOR_IP>:<PORT>/cluster
|
|||
|
||||
## My queries are returning empty results
|
||||
|
||||
You can check <BROKER_IP>:<PORT>/druid/v2/datasources/<YOUR_DATASOURCE> for the dimensions and metrics that have been created for your datasource. Make sure that the name of the aggregators you use in your query match one of these metrics. Also make sure that the query interval you specify match a valid time range where data exists.
|
||||
You can check `<BROKER_IP>:<PORT>/druid/v2/datasources/<YOUR_DATASOURCE>` for the dimensions and metrics that have been created for your datasource. Make sure that the name of the aggregators you use in your query match one of these metrics. Also make sure that the query interval you specify match a valid time range where data exists.
|
||||
|
||||
## More information
|
||||
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# MySQL Database
|
||||
MySQL is an external dependency of Druid. We use it to store various metadata about the system, but not to store the actual data. There are a number of tables used for various purposes described below.
|
||||
|
||||
Segments Table
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Sort groupBy Query Results
|
||||
The orderBy field provides the functionality to sort and limit the set of results from a groupBy query. If you group by a single dimension and are ordering by a single metric, we highly recommend using [TopN Queries](TopNQuery.html) instead. The performance will be substantially better. Available options are:
|
||||
|
||||
### DefaultLimitSpec
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Post-Aggregations
|
||||
Post-aggregations are specifications of processing that should happen on aggregated values as they come out of Druid. If you include a post aggregation as part of a query, make sure to include all aggregators the post-aggregator requires.
|
||||
|
||||
There are several post-aggregators available.
|
||||
|
@ -63,6 +64,31 @@ Example JavaScript aggregator:
|
|||
"function": "function(delta, total) { return 100 * Math.abs(delta) / total; }"
|
||||
}
|
||||
```
|
||||
### `hyperUniqueCardinality` post-aggregator
|
||||
|
||||
The hyperUniqueCardinality post aggregator is used to wrap a hyperUnique object such that it can be used in post aggregations.
|
||||
|
||||
```json
|
||||
{ "type" : "hyperUniqueCardinality", "fieldName" : <the name field value of the hyperUnique aggregator>}
|
||||
```
|
||||
|
||||
It can be used in a sample calculation as so:
|
||||
|
||||
```json
|
||||
"aggregations" : [{
|
||||
{"type" : "count", "name" : "rows"},
|
||||
{"type" : "hyperUnique", "name" : "unique_users", "fieldName" : "uniques"}
|
||||
}],
|
||||
"postAggregations" : {
|
||||
"type" : "arithmetic",
|
||||
"name" : "average_users_per_row",
|
||||
"fn" : "/",
|
||||
"fields" : [
|
||||
{ "type" : "hyperUniqueCardinality", "fieldName" : "unique_users" },
|
||||
{ "type" : "fieldAccess", "name" : "rows", "fieldName" : "rows" }
|
||||
]
|
||||
}
|
||||
```
|
||||
|
||||
|
||||
### Example Usage
|
||||
|
|
|
@ -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.72"]
|
||||
|
||||
|
||||
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.72","io.druid.extensions:druid-kafka-seven:0.6.72"]
|
||||
|
||||
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
|
||||
-----------
|
||||
|
||||
|
@ -132,17 +117,3 @@ 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 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.
|
||||
|
||||
|
||||
|
||||
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.
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Configuring Rules for Coordinator Nodes
|
||||
Note: It is recommended that the coordinator console is used to configure rules. However, the coordinator node does have HTTP endpoints to programmatically configure rules.
|
||||
|
||||
Load Rules
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Search Queries
|
||||
A search query returns dimension values that match the search specification.
|
||||
|
||||
```json
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Refining Search Queries
|
||||
Search query specs define how a "match" is defined between a search value and a dimension value. The available search query specs are:
|
||||
|
||||
InsensitiveContainsSearchQuerySpec
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Segment Metadata Queries
|
||||
Segment metadata queries return per segment information about:
|
||||
|
||||
* Cardinality of all columns in the segment
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Tasks
|
||||
Tasks are run on middle managers and always operate on a single data source.
|
||||
|
||||
There are several different types of tasks.
|
||||
|
@ -8,7 +9,7 @@ There are several different types of tasks.
|
|||
Segment Creation Tasks
|
||||
----------------------
|
||||
|
||||
#### Index Task
|
||||
### Index Task
|
||||
|
||||
The Index Task is a simpler variation of the Index Hadoop task that is designed to be used for smaller data sets. The task executes within the indexing service and does not require an external Hadoop setup to use. The grammar of the index task is as follows:
|
||||
|
||||
|
@ -50,15 +51,15 @@ The Index Task is a simpler variation of the Index Hadoop task that is designed
|
|||
|--------|-----------|---------|
|
||||
|type|The task type, this should always be "index".|yes|
|
||||
|id|The task ID.|no|
|
||||
|granularitySpec|See [granularitySpec](Tasks.html)|yes|
|
||||
|spatialDimensions|Dimensions to build spatial indexes over. See [Spatial-Indexing](Spatial-Indexing.html)|no|
|
||||
|granularitySpec|Specifies the segment chunks that the task will process. `type` is always "uniform"; `gran` sets the granularity of the chunks ("DAY" means all segments containing timestamps in the same day, while `intervals` sets the interval that the chunks will cover.|yes|
|
||||
|spatialDimensions|Dimensions to build spatial indexes over. See [Geographic Queries](GeographicQueries.html).|no|
|
||||
|aggregators|The metrics to aggregate in the data set. For more info, see [Aggregations](Aggregations.html)|yes|
|
||||
|indexGranularity|The rollup granularity for timestamps.|no|
|
||||
|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|no|
|
||||
|firehose|The input source of data. For more info, see [Firehose](Firehose.html)|yes|
|
||||
|rowFlushBoundary|Used in determining when intermediate persist should occur to disk.|no|
|
||||
|
||||
#### Index Hadoop Task
|
||||
### Index Hadoop Task
|
||||
|
||||
The Hadoop Index Task is used to index larger data sets that require the parallelization and processing power of a Hadoop cluster.
|
||||
|
||||
|
@ -78,11 +79,11 @@ The Hadoop Index Task is used to index larger data sets that require the paralle
|
|||
|
||||
The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopBatchIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `updaterJobSpec`. The Indexing Service takes care of setting these fields internally.
|
||||
|
||||
##### Using your own Hadoop distribution
|
||||
#### Using your own Hadoop distribution
|
||||
|
||||
Druid is compiled against Apache hadoop-core 1.0.3. However, if you happen to use a different flavor of hadoop that is API compatible with hadoop-core 1.0.3, you should only have to change the hadoopCoordinates property to point to the maven artifact used by your distribution.
|
||||
|
||||
##### Resolving dependency conflicts running HadoopIndexTask
|
||||
#### Resolving dependency conflicts running HadoopIndexTask
|
||||
|
||||
Currently, the HadoopIndexTask creates a single classpath to run the HadoopDruidIndexerJob, which can lead to version conflicts between various dependencies of Druid, extension modules, and Hadoop's own dependencies.
|
||||
|
||||
|
@ -90,7 +91,7 @@ The Hadoop index task will put Druid's dependencies first on the classpath, foll
|
|||
|
||||
If you are having trouble with any extensions in HadoopIndexTask, it may be the case that Druid, or one of its dependencies, depends on a different version of a library than what you are using as part of your extensions, but Druid's version overrides the one in your extension. In that case you probably want to build your own Druid version and override the offending library by adding an explicit dependency to the pom.xml of each druid sub-module that depends on it.
|
||||
|
||||
#### Realtime Index Task
|
||||
### Realtime Index Task
|
||||
|
||||
The indexing service can also run real-time tasks. These tasks effectively transform a middle manager into a real-time node. We introduced real-time tasks as a way to programmatically add new real-time data sources without needing to manually add nodes. The grammar for the real-time task is as follows:
|
||||
|
||||
|
@ -151,46 +152,24 @@ The indexing service can also run real-time tasks. These tasks effectively trans
|
|||
"intermediatePersistPeriod": "PT10m"
|
||||
},
|
||||
"windowPeriod": "PT10m",
|
||||
"segmentGranularity": "hour",
|
||||
"rejectionPolicy": {
|
||||
"type": "messageTime"
|
||||
}
|
||||
"segmentGranularity": "hour"
|
||||
}
|
||||
```
|
||||
|
||||
Id:
|
||||
The ID of the task. Not required.
|
||||
|
||||
Resource:
|
||||
A JSON object used for high availability purposes. Not required.
|
||||
|
||||
|Field|Type|Description|Required|
|
||||
|-----|----|-----------|--------|
|
||||
|id|String|The ID of the task.|No|
|
||||
|Resource|JSON object|Used for high availability purposes.|No|
|
||||
|availabilityGroup|String|An uniqueness identifier for the task. Tasks with the same availability group will always run on different middle managers. Used mainly for replication. |yes|
|
||||
|requiredCapacity|Integer|How much middle manager capacity this task will take.|yes|
|
||||
|
||||
Schema:
|
||||
See [Schema](Realtime.html).
|
||||
For schema, fireDepartmentConfig, windowPeriod, segmentGranularity, and rejectionPolicy, see the [realtime-ingestion doc](Realtime-ingestion.html). For firehose configuration, see [Firehose](Firehose.html).
|
||||
|
||||
Fire Department Config:
|
||||
See [Config](Realtime.html).
|
||||
|
||||
Firehose:
|
||||
See [Firehose](Firehose.html).
|
||||
|
||||
Window Period:
|
||||
See [Realtime](Realtime.html).
|
||||
|
||||
Segment Granularity:
|
||||
See [Realtime](Realtime.html).
|
||||
|
||||
Rejection Policy:
|
||||
See [Realtime](Realtime.html).
|
||||
|
||||
Segment Merging Tasks
|
||||
---------------------
|
||||
|
||||
#### Append Task
|
||||
### Append Task
|
||||
|
||||
Append tasks append a list of segments together into a single segment (one after the other). The grammar is:
|
||||
|
||||
|
@ -202,7 +181,7 @@ Append tasks append a list of segments together into a single segment (one after
|
|||
}
|
||||
```
|
||||
|
||||
#### Merge Task
|
||||
### Merge Task
|
||||
|
||||
Merge tasks merge a list of segments together. Any common timestamps are merged. The grammar is:
|
||||
|
||||
|
@ -217,7 +196,7 @@ Merge tasks merge a list of segments together. Any common timestamps are merged.
|
|||
Segment Destroying Tasks
|
||||
------------------------
|
||||
|
||||
#### Delete Task
|
||||
### Delete Task
|
||||
|
||||
Delete tasks create empty segments with no data. The grammar is:
|
||||
|
||||
|
@ -229,7 +208,7 @@ Delete tasks create empty segments with no data. The grammar is:
|
|||
}
|
||||
```
|
||||
|
||||
#### Kill Task
|
||||
### Kill Task
|
||||
|
||||
Kill tasks delete all information about a segment and removes it from deep storage. Killable segments must be disabled (used==0) in the Druid segment table. The available grammar is:
|
||||
|
||||
|
@ -244,7 +223,7 @@ Kill tasks delete all information about a segment and removes it from deep stora
|
|||
Misc. Tasks
|
||||
-----------
|
||||
|
||||
#### Version Converter Task
|
||||
### Version Converter Task
|
||||
|
||||
These tasks convert segments from an existing older index version to the latest index version. The available grammar is:
|
||||
|
||||
|
@ -258,7 +237,7 @@ These tasks convert segments from an existing older index version to the latest
|
|||
}
|
||||
```
|
||||
|
||||
#### Noop Task
|
||||
### Noop Task
|
||||
|
||||
These tasks start, sleep for a time and are used only for testing. The available grammar is:
|
||||
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Time Boundary Queries
|
||||
Time boundary queries return the earliest and latest data points of a data set. The grammar is:
|
||||
|
||||
```json
|
||||
|
|
|
@ -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|
|
||||
|
||||
|
|
|
@ -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.72-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.72
|
||||
```
|
||||
|
||||
You should see a bunch of files:
|
||||
|
@ -80,7 +80,7 @@ Let's start doing stuff. You can start a Druid [Realtime](Realtime.html) node by
|
|||
|
||||
Select "wikipedia".
|
||||
|
||||
Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below.
|
||||
Note that the first time you start the example, it may take some extra time due to its fetching various dependencies. Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below.
|
||||
|
||||
```
|
||||
2013-09-04 19:33:11,922 INFO [main] org.eclipse.jetty.server.AbstractConnector - Started SelectChannelConnector@0.0.0.0:8083
|
||||
|
@ -118,7 +118,7 @@ Select "wikipedia" once again. This script issues [GroupByQueries](GroupByQuery.
|
|||
|
||||
This is a **groupBy** query, which you may be familiar with from SQL. We are grouping, or aggregating, via the `dimensions` field: `["page"]`. We are **filtering** via the `namespace` dimension, to only look at edits on `articles`. Our **aggregations** are what we are calculating: a count of the number of data rows, and a count of the number of edits that have occurred.
|
||||
|
||||
The result looks something like this:
|
||||
The result looks something like this (when it's prettified):
|
||||
|
||||
```json
|
||||
[
|
||||
|
@ -323,13 +323,13 @@ 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).
|
||||
|
||||
Additional Information
|
||||
----------------------
|
||||
|
||||
This tutorial is merely showcasing a small fraction of what Druid can do. If you are interested in more information about Druid, including setting up a more sophisticated Druid cluster, please read the other links in our wiki.
|
||||
This tutorial is merely showcasing a small fraction of what Druid can do. If you are interested in more information about Druid, including setting up a more sophisticated Druid cluster, read more of the Druid documentation and the blogs found on druid.io.
|
||||
|
||||
And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](https://groups.google.com/forum/#!forum/druid-development).
|
||||
|
|
|
@ -14,6 +14,8 @@ Before we start digging into how to query Druid, make sure you've gone through t
|
|||
|
||||
Let's start up a simple Druid cluster so we can query all the things.
|
||||
|
||||
Note: If Zookeeper and MySQL aren't running, you'll have to start them again as described in [The Druid Cluster](Tutorial%3A-The-Druid-Cluster.html).
|
||||
|
||||
To start a Coordinator node:
|
||||
|
||||
```bash
|
||||
|
|
|
@ -66,6 +66,8 @@ There are five data points spread across the day of 2013-08-31. Talk about big d
|
|||
|
||||
In order to ingest and query this data, we are going to need to run a historical node, a coordinator node, and an indexing service to run the batch ingestion.
|
||||
|
||||
Note: If Zookeeper and MySQL aren't running, you'll have to start them again as described in [The Druid Cluster](Tutorial%3A-The-Druid-Cluster.html).
|
||||
|
||||
#### Starting a Local Indexing Service
|
||||
|
||||
The simplest indexing service we can start up is to run an [overlord](Indexing-Service.html) node in local mode. You can do so by issuing:
|
||||
|
|
|
@ -45,9 +45,9 @@ 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](Firehose.html) 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).
|
||||
The following quick-start instructions for booting a Zookeeper and then Kafka cluster were taken from the [Kafka website](http://kafka.apache.org/07/quickstart.html).
|
||||
|
||||
1. Download Apache Kafka 0.7.2 from [http://kafka.apache.org/downloads.html](http://kafka.apache.org/downloads.html)
|
||||
|
||||
|
@ -160,13 +160,15 @@ You should be comfortable starting Druid nodes at this point. If not, it may be
|
|||
"segmentGranularity": "hour",
|
||||
"basePersistDirectory": "\/tmp\/realtime\/basePersist",
|
||||
"rejectionPolicy": {
|
||||
"type": "none"
|
||||
"type": "test"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
```
|
||||
|
||||
Note: This config uses a "test" rejection policy which will accept all events and timely hand off, however, we strongly recommend you do not use this in production. Using this rejection policy, segments for events for the same time range will be overridden.
|
||||
|
||||
3. Let's copy and paste some data into the Kafka console producer
|
||||
|
||||
```json
|
||||
|
@ -207,9 +209,9 @@ Batch Ingestion
|
|||
---------------
|
||||
Druid is designed for large data volumes, and most real-world data sets require batch indexing be done through a Hadoop job.
|
||||
|
||||
The setup for a single node, 'standalone' Hadoop cluster is available [here](http://hadoop.apache.org/docs/stable/single_node_setup.html).
|
||||
For this tutorial, we used [Hadoop 1.0.3](https://archive.apache.org/dist/hadoop/core/hadoop-1.0.3/). There are many pages on the Internet showing how to set up a single-node (standalone) Hadoop cluster, which is all that's needed for this example.
|
||||
|
||||
For the purposes of this tutorial, we are going to use our very small and simple Wikipedia data set. This data can directly be ingested via other means as shown in the previous [tutorial](Tutorial%3A-Loading-Your-Data-Part-1), but we are going to use Hadoop here for demonstration purposes.
|
||||
For the purposes of this tutorial, we are going to use our very small and simple Wikipedia data set. This data can directly be ingested via other means as shown in the previous [tutorial](Tutorial%3A-Loading-Your-Data-Part-1.html), but we are going to use Hadoop here for demonstration purposes.
|
||||
|
||||
Our data is located at:
|
||||
|
||||
|
@ -227,10 +229,12 @@ The following events should exist in the file:
|
|||
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
||||
```
|
||||
|
||||
#### Setup a Druid Cluster
|
||||
#### Set Up a Druid Cluster
|
||||
|
||||
To index the data, we are going to need an indexing service, a historical node, and a coordinator node.
|
||||
|
||||
Note: If Zookeeper and MySQL aren't running, you'll have to start them again as described in [The Druid Cluster](Tutorial%3A-The-Druid-Cluster.html).
|
||||
|
||||
To start the Indexing Service:
|
||||
|
||||
```bash
|
||||
|
@ -305,7 +309,7 @@ Examining the contents of the file, you should find:
|
|||
}
|
||||
```
|
||||
|
||||
If you are curious about what all this configuration means, see [here](Task.html).
|
||||
If you are curious about what all this configuration means, see [here](Tasks.html).
|
||||
|
||||
To submit the task:
|
||||
|
||||
|
|
|
@ -7,13 +7,13 @@ Welcome back! In our first [tutorial](Tutorial%3A-A-First-Look-at-Druid.html), w
|
|||
|
||||
This tutorial will hopefully answer these questions!
|
||||
|
||||
In this tutorial, we will set up other types of Druid nodes as well as and external dependencies for a fully functional Druid cluster. The architecture of Druid is very much like the [Megazord](http://www.youtube.com/watch?v=7mQuHh1X4H4) from the popular 90s show Mighty Morphin' Power Rangers. Each Druid node has a specific purpose and the nodes come together to form a fully functional system.
|
||||
In this tutorial, we will set up other types of Druid nodes and external dependencies for a fully functional Druid cluster. The architecture of Druid is very much like the [Megazord](http://www.youtube.com/watch?v=7mQuHh1X4H4) from the popular 90s show Mighty Morphin' Power Rangers. Each Druid node has a specific purpose and the nodes come together to form a fully functional system.
|
||||
|
||||
## Downloading Druid
|
||||
|
||||
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.72-bin.tar.gz)
|
||||
|
||||
and untar the contents within by issuing:
|
||||
|
||||
|
@ -32,9 +32,9 @@ For deep storage, we have made a public S3 bucket (static.druid.io) available wh
|
|||
|
||||
#### Setting up MySQL
|
||||
|
||||
1. If you don't already have it, download MySQL Community Server here: [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/)
|
||||
2. Install MySQL
|
||||
3. Create a druid user and database
|
||||
1. If you don't already have it, download MySQL Community Server here: [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/).
|
||||
2. Install MySQL.
|
||||
3. Create a druid user and database.
|
||||
|
||||
```bash
|
||||
mysql -u root
|
||||
|
@ -88,7 +88,7 @@ Metrics (things to aggregate over):
|
|||
|
||||
## The Cluster
|
||||
|
||||
Let's start up a few nodes and download our data. First things though, let's make sure we have config directory where we will store configs for our various nodes:
|
||||
Let's start up a few nodes and download our data. First, let's make sure we have configs in the config directory for our various nodes. Issue the following from the Druid home directory:
|
||||
|
||||
```
|
||||
ls config
|
||||
|
@ -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.72"]
|
||||
|
||||
# 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.72","io.druid.extensions:druid-kafka-seven:0.6.72"]
|
||||
|
||||
# Change this config to db to hand off to the rest of the Druid cluster
|
||||
druid.publish.type=noop
|
||||
|
@ -251,6 +251,9 @@ druid.publish.type=noop
|
|||
# druid.db.connector.password=diurd
|
||||
|
||||
druid.processing.buffer.sizeBytes=100000000
|
||||
druid.processing.numThreads=1
|
||||
|
||||
druid.monitoring.monitors=["io.druid.segment.realtime.RealtimeMetricsMonitor"]
|
||||
```
|
||||
|
||||
Next Steps
|
||||
|
|
|
@ -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.72-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.72
|
||||
```
|
||||
|
||||
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.72-bin.tar.gz.
|
||||
Download this bad boy to a directory of your choosing.
|
||||
|
||||
You can extract the awesomeness within by issuing:
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Versioning Druid
|
||||
This page discusses how we do versioning and provides information on our stable releases.
|
||||
|
||||
Versioning Strategy
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# ZooKeeper
|
||||
Druid uses [ZooKeeper](http://zookeeper.apache.org/) (ZK) for management of current cluster state. The operations that happen over ZK are
|
||||
|
||||
1. [Coordinator](Coordinator.html) leader election
|
||||
|
|
|
@ -54,3 +54,7 @@
|
|||
.doc-content table code {
|
||||
background-color: transparent;
|
||||
}
|
||||
|
||||
td, th {
|
||||
padding: 5px;
|
||||
}
|
||||
|
|
|
@ -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,11 +30,10 @@ 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
|
||||
* "Data Formats":./Data_formats.html
|
||||
* "Ingestion FAQ":./Ingestion-FAQ.html
|
||||
|
||||
h2. Querying
|
||||
|
@ -38,15 +44,15 @@ h2. Querying
|
|||
** "Granularities":./Granularities.html
|
||||
** "DimensionSpecs":./DimensionSpecs.html
|
||||
* Query Types
|
||||
** "GroupByQuery":./GroupByQuery.html
|
||||
** "GroupBy":./GroupByQuery.html
|
||||
*** "OrderBy":./OrderBy.html
|
||||
*** "Having":./Having.html
|
||||
** "SearchQuery":./SearchQuery.html
|
||||
** "Search":./SearchQuery.html
|
||||
*** "SearchQuerySpec":./SearchQuerySpec.html
|
||||
** "SegmentMetadataQuery":./SegmentMetadataQuery.html
|
||||
** "TimeBoundaryQuery":./TimeBoundaryQuery.html
|
||||
** "TimeseriesQuery":./TimeseriesQuery.html
|
||||
** "TopNQuery":./TopNQuery.html
|
||||
** "Segment Metadata":./SegmentMetadataQuery.html
|
||||
** "Time Boundary":./TimeBoundaryQuery.html
|
||||
** "Timeseries":./TimeseriesQuery.html
|
||||
** "TopN":./TopNQuery.html
|
||||
*** "TopNMetricSpec":./TopNMetricSpec.html
|
||||
|
||||
h2. Architecture
|
||||
|
|
|
@ -53,7 +53,7 @@
|
|||
"segmentGranularity": "hour",
|
||||
"basePersistDirectory": "\/tmp\/realtime\/basePersist",
|
||||
"rejectionPolicy": {
|
||||
"type": "messageTime"
|
||||
"type": "test"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,6 +43,6 @@
|
|||
"windowPeriod" : "PT5m",
|
||||
"segmentGranularity":"hour",
|
||||
"basePersistDirectory" : "/tmp/realtime/basePersist",
|
||||
"rejectionPolicy": { "type": "messageTime" }
|
||||
"rejectionPolicy": { "type": "test" }
|
||||
}
|
||||
}]
|
||||
|
|
|
@ -55,8 +55,6 @@ JAVA_ARGS="${JAVA_ARGS} -Ddruid.realtime.specFile=${SPEC_FILE}"
|
|||
|
||||
DRUID_CP=${EXAMPLE_LOC}
|
||||
#For a pull
|
||||
DRUID_CP=${DRUID_CP}:`ls ${SCRIPT_DIR}/../target/druid-examples-*-selfcontained.jar`
|
||||
DRUID_CP=${DRUID_CP}:`ls ${SCRIPT_DIR}/../../services/target/druid-services-*-selfcontained.jar`
|
||||
DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/../config/realtime
|
||||
#For the kit
|
||||
DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/lib/*
|
||||
|
|
|
@ -3,3 +3,7 @@ druid.service=broker
|
|||
druid.port=8080
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
# Change these to make Druid faster
|
||||
druid.processing.buffer.sizeBytes=100000000
|
||||
druid.processing.numThreads=1
|
||||
|
|
|
@ -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.72"]
|
||||
|
||||
# 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.72","io.druid.extensions:druid-kafka-seven:0.6.72","io.druid.extensions:druid-rabbitmq:0.6.72"]
|
||||
|
||||
# Change this config to db to hand off to the rest of the Druid cluster
|
||||
druid.publish.type=noop
|
||||
|
@ -15,3 +15,6 @@ druid.publish.type=noop
|
|||
# druid.db.connector.password=diurd
|
||||
|
||||
druid.processing.buffer.sizeBytes=100000000
|
||||
druid.processing.numThreads=1
|
||||
|
||||
druid.monitoring.monitors=["io.druid.segment.realtime.RealtimeMetricsMonitor"]
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.53-SNAPSHOT</version>
|
||||
<version>0.6.73-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.53-SNAPSHOT</version>
|
||||
<version>0.6.73-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -18,8 +18,7 @@
|
|||
~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>io.druid.extensions</groupId>
|
||||
<artifactId>druid-hll</artifactId>
|
||||
|
@ -29,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.53-SNAPSHOT</version>
|
||||
<version>0.6.73-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.53-SNAPSHOT</version>
|
||||
<version>0.6.73-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -0,0 +1,359 @@
|
|||
/*
|
||||
* 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.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.query.aggregation.hyperloglog.HyperLogLogCollector;
|
||||
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.nio.ByteBuffer;
|
||||
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 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);
|
||||
if (!config.getSegmentGranularIntervals().isPresent()) {
|
||||
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, HyperLogLogCollector> 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, HyperLogLogCollector> builder = ImmutableMap.builder();
|
||||
for (final Interval bucketInterval : intervals.get()) {
|
||||
builder.put(bucketInterval, HyperLogLogCollector.makeLatestCollector());
|
||||
}
|
||||
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, HyperLogLogCollector.makeLatestCollector());
|
||||
}
|
||||
} 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)
|
||||
.add(
|
||||
hashFunction.hashBytes(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey))
|
||||
.asBytes()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(Context context) throws IOException, InterruptedException
|
||||
{
|
||||
setup(context);
|
||||
|
||||
while (context.nextKeyValue()) {
|
||||
map(context.getCurrentKey(), context.getCurrentValue(), context);
|
||||
}
|
||||
|
||||
for (Map.Entry<Interval, HyperLogLogCollector> entry : hyperLogLogs.entrySet()) {
|
||||
context.write(
|
||||
new LongWritable(entry.getKey().getStartMillis()),
|
||||
new BytesWritable(entry.getValue().toByteArray())
|
||||
);
|
||||
}
|
||||
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
|
||||
{
|
||||
HyperLogLogCollector aggregate = HyperLogLogCollector.makeLatestCollector();
|
||||
for (BytesWritable value : values) {
|
||||
aggregate.fold(ByteBuffer.wrap(value.getBytes(), 0, value.getLength()));
|
||||
}
|
||||
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,
|
||||
new Double(aggregate.estimateCardinality()).longValue()
|
||||
);
|
||||
}
|
||||
finally {
|
||||
Closeables.close(out, false);
|
||||
}
|
||||
}
|
||||
|
||||
private byte[] getDataBytes(BytesWritable writable)
|
||||
{
|
||||
byte[] rv = new byte[writable.getLength()];
|
||||
System.arraycopy(writable.getBytes(), 0, rv, 0, writable.getLength());
|
||||
return rv;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(Context context)
|
||||
throws IOException, InterruptedException
|
||||
{
|
||||
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,10 +38,13 @@ 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;
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -76,7 +77,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 +107,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 +115,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 +162,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 +184,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 +210,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 +225,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 +261,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 +299,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 +331,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 +367,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++;
|
||||
}
|
||||
|
@ -433,15 +417,19 @@ public class DeterminePartitionsJob implements Jobby
|
|||
}
|
||||
|
||||
public static class DeterminePartitionsDimSelectionPartitioner
|
||||
extends Partitioner<BytesWritable, Text>
|
||||
extends Partitioner<BytesWritable, Text> implements Configurable
|
||||
{
|
||||
private Configuration config;
|
||||
|
||||
@Override
|
||||
public int getPartition(BytesWritable bytesWritable, Text text, int numPartitions)
|
||||
{
|
||||
final ByteBuffer bytes = ByteBuffer.wrap(bytesWritable.getBytes());
|
||||
bytes.position(4); // Skip length added by SortableBytes
|
||||
final int index = bytes.getInt();
|
||||
|
||||
if (config.get("mapred.job.tracker").equals("local")) {
|
||||
return index % numPartitions;
|
||||
} else {
|
||||
if (index >= numPartitions) {
|
||||
throw new ISE(
|
||||
"Not enough partitions, index[%,d] >= numPartitions[%,d]. Please increase the number of reducers to the index size or check your config & settings!",
|
||||
|
@ -449,9 +437,22 @@ public class DeterminePartitionsJob implements Jobby
|
|||
numPartitions
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
return index;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConf()
|
||||
{
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration config)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
}
|
||||
|
||||
private static abstract class DeterminePartitionsDimSelectionBaseReducer
|
||||
|
@ -741,7 +742,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();
|
||||
}
|
||||
|
@ -483,15 +485,18 @@ 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())
|
||||
Optional<Set<Interval>> intervals = getSegmentGranularIntervals();
|
||||
if (intervals.isPresent()) {
|
||||
return Optional.of(
|
||||
(Iterable<Bucket>) FunctionalIterable
|
||||
.create(intervals.get())
|
||||
.transformCat(
|
||||
new Function<Interval, Iterable<Bucket>>()
|
||||
{
|
||||
|
@ -520,7 +525,11 @@ public class HadoopDruidIndexerConfig
|
|||
);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
} else {
|
||||
return Optional.absent();
|
||||
}
|
||||
}
|
||||
|
||||
/******************************************
|
||||
|
@ -532,15 +541,14 @@ public class HadoopDruidIndexerConfig
|
|||
*
|
||||
* @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,64 +66,23 @@ 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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (failedMessage == null) {
|
||||
jobs.add(new Jobby()
|
||||
{
|
||||
@Override
|
||||
public boolean run()
|
||||
{
|
||||
publishedSegments = IndexGeneratorJob.getPublishedSegments(config);
|
||||
return true;
|
||||
}
|
||||
});
|
||||
|
||||
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) {
|
||||
throw new IllegalStateException("Job hasn't run yet. No segments have been published yet.");
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@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,22 +44,29 @@ public class UniformGranularitySpec implements GranularitySpec
|
|||
@JsonProperty("intervals") List<Interval> inputIntervals
|
||||
)
|
||||
{
|
||||
this.granularity = granularity;
|
||||
if (inputIntervals != null) {
|
||||
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);
|
||||
} else {
|
||||
this.inputIntervals = null;
|
||||
this.wrappedSpec = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSet<Interval> bucketIntervals()
|
||||
public Optional<SortedSet<Interval>> bucketIntervals()
|
||||
{
|
||||
if (wrappedSpec == null) {
|
||||
return Optional.absent();
|
||||
} else {
|
||||
return wrappedSpec.bucketIntervals();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Interval> bucketInterval(DateTime dt)
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 HashedPartitionsSpec extends AbstractPartitionsSpec
|
||||
{
|
||||
@JsonCreator
|
||||
public HashedPartitionsSpec(
|
||||
@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);
|
||||
}
|
||||
}
|
|
@ -19,69 +19,34 @@
|
|||
|
||||
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),
|
||||
@JsonSubTypes.Type(name = "hashed", value = HashedPartitionsSpec.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,43 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
// for backward compatibility
|
||||
@Deprecated
|
||||
public class RandomPartitionsSpec extends HashedPartitionsSpec
|
||||
{
|
||||
@JsonCreator
|
||||
public RandomPartitionsSpec(
|
||||
@JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize,
|
||||
@JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize,
|
||||
@JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped
|
||||
)
|
||||
{
|
||||
super(targetPartitionSize, maxPartitionSize, assumeGrouped);
|
||||
}
|
||||
}
|
|
@ -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,11 +100,14 @@ 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()) {
|
||||
Optional<Set<Interval>> optionalIntervals = config.getSegmentGranularIntervals();
|
||||
if (optionalIntervals.isPresent()) {
|
||||
for (Interval segmentInterval : optionalIntervals.get()) {
|
||||
for (Interval dataInterval : dataGranularity.getIterable(segmentInterval)) {
|
||||
intervals.add(dataInterval);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Path betaInput = new Path(inputPath);
|
||||
FileSystem fs = betaInput.getFileSystem(job.getConfiguration());
|
||||
|
|
|
@ -24,7 +24,10 @@ import com.google.common.base.Throwables;
|
|||
import com.google.common.collect.Lists;
|
||||
import io.druid.db.DbConnectorConfig;
|
||||
import io.druid.indexer.granularity.UniformGranularitySpec;
|
||||
import io.druid.indexer.partitions.HashedPartitionsSpec;
|
||||
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 +68,7 @@ public class HadoopDruidIndexerConfigTest
|
|||
Assert.assertEquals(
|
||||
"getIntervals",
|
||||
Lists.newArrayList(new Interval("2012-01-01/P1D")),
|
||||
granularitySpec.getIntervals()
|
||||
granularitySpec.getIntervals().get()
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
|
@ -99,7 +102,7 @@ public class HadoopDruidIndexerConfigTest
|
|||
Assert.assertEquals(
|
||||
"getIntervals",
|
||||
Lists.newArrayList(new Interval("2012-02-01/P1D")),
|
||||
granularitySpec.getIntervals()
|
||||
granularitySpec.getIntervals().get()
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
|
@ -167,15 +170,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 +216,10 @@ public class HadoopDruidIndexerConfigTest
|
|||
150
|
||||
);
|
||||
|
||||
Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof SingleDimensionPartitionsSpec);
|
||||
Assert.assertEquals(
|
||||
"getPartitionDimension",
|
||||
partitionsSpec.getPartitionDimension(),
|
||||
((SingleDimensionPartitionsSpec)partitionsSpec).getPartitionDimension(),
|
||||
"foo"
|
||||
);
|
||||
}
|
||||
|
@ -259,9 +262,10 @@ public class HadoopDruidIndexerConfigTest
|
|||
150
|
||||
);
|
||||
|
||||
Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof SingleDimensionPartitionsSpec);
|
||||
Assert.assertEquals(
|
||||
"getPartitionDimension",
|
||||
partitionsSpec.getPartitionDimension(),
|
||||
((SingleDimensionPartitionsSpec)partitionsSpec).getPartitionDimension(),
|
||||
"foo"
|
||||
);
|
||||
}
|
||||
|
@ -307,9 +311,10 @@ public class HadoopDruidIndexerConfigTest
|
|||
200
|
||||
);
|
||||
|
||||
Assert.assertTrue("partitionsSpec" , partitionsSpec instanceof SingleDimensionPartitionsSpec);
|
||||
Assert.assertEquals(
|
||||
"getPartitionDimension",
|
||||
partitionsSpec.getPartitionDimension(),
|
||||
((SingleDimensionPartitionsSpec)partitionsSpec).getPartitionDimension(),
|
||||
"foo"
|
||||
);
|
||||
}
|
||||
|
@ -420,7 +425,6 @@ public class HadoopDruidIndexerConfigTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void shouldMakeHDFSCompliantSegmentOutputPath()
|
||||
{
|
||||
|
@ -497,4 +501,92 @@ public class HadoopDruidIndexerConfigTest
|
|||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHashedPartitionsSpec() throws Exception{
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
cfg = jsonReadWriteRead(
|
||||
"{"
|
||||
+ "\"partitionsSpec\":{"
|
||||
+ " \"targetPartitionSize\":100,"
|
||||
+ " \"type\":\"hashed\""
|
||||
+ " }"
|
||||
+ "}",
|
||||
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 HashedPartitionsSpec);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.53-SNAPSHOT</version>
|
||||
<version>0.6.73-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -32,6 +32,7 @@ import io.druid.curator.discovery.ServerDiscoverySelector;
|
|||
import io.druid.indexing.common.RetryPolicy;
|
||||
import io.druid.indexing.common.RetryPolicyFactory;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import org.jboss.netty.channel.ChannelException;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -94,6 +95,7 @@ public class RemoteTaskActionClient implements TaskActionClient
|
|||
}
|
||||
catch (Exception e) {
|
||||
Throwables.propagateIfInstanceOf(e.getCause(), IOException.class);
|
||||
Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class);
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -105,7 +107,7 @@ public class RemoteTaskActionClient implements TaskActionClient
|
|||
|
||||
return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference());
|
||||
}
|
||||
catch (IOException e) {
|
||||
catch (IOException | ChannelException e) {
|
||||
log.warn(e, "Exception submitting action for task[%s]", task.getId());
|
||||
|
||||
final Duration delay = retryPolicy.getAndIncrementRetryDelay();
|
||||
|
|
|
@ -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 determine configuration job...");
|
||||
if (job.run()) {
|
||||
return HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(HadoopDruidIndexerConfigBuilder.toSchema(config));
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -109,8 +109,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 +137,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!");
|
||||
}
|
||||
|
|
|
@ -109,7 +109,7 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
|
||||
@JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig,
|
||||
@JsonProperty("windowPeriod") Period windowPeriod,
|
||||
@JsonProperty("maxPendingPersists") int maxPendingPersists,
|
||||
@JsonProperty("maxPendingPersists") Integer maxPendingPersists,
|
||||
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity,
|
||||
@JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory
|
||||
)
|
||||
|
@ -139,7 +139,7 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
this.firehoseFactory = firehoseFactory;
|
||||
this.fireDepartmentConfig = fireDepartmentConfig;
|
||||
this.windowPeriod = windowPeriod;
|
||||
this.maxPendingPersists = (maxPendingPersists == 0)
|
||||
this.maxPendingPersists = (maxPendingPersists == null)
|
||||
? RealtimePlumberSchool.DEFAULT_MAX_PENDING_PERSISTS
|
||||
: maxPendingPersists;
|
||||
this.segmentGranularity = segmentGranularity;
|
||||
|
@ -398,6 +398,12 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
return windowPeriod;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMaxPendingPersists()
|
||||
{
|
||||
return maxPendingPersists;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public IndexGranularity getSegmentGranularity()
|
||||
{
|
||||
|
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* 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.indexing.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.indexing.common.TaskLock;
|
||||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.indexing.common.TaskToolbox;
|
||||
import io.druid.indexing.common.actions.SegmentListUnusedAction;
|
||||
import io.druid.indexing.common.actions.SegmentMetadataUpdateAction;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class RestoreTask extends AbstractFixedIntervalTask
|
||||
{
|
||||
private static final Logger log = new Logger(RestoreTask.class);
|
||||
|
||||
public RestoreTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
super(
|
||||
TaskUtils.makeId(id, "restore", dataSource, interval),
|
||||
dataSource,
|
||||
interval
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "restore";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
// Confirm we have a lock (will throw if there isn't exactly one element)
|
||||
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
|
||||
|
||||
if (!myLock.getDataSource().equals(getDataSource())) {
|
||||
throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
|
||||
}
|
||||
|
||||
if (!myLock.getInterval().equals(getInterval())) {
|
||||
throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getInterval());
|
||||
}
|
||||
|
||||
// List unused segments
|
||||
final List<DataSegment> unusedSegments = toolbox
|
||||
.getTaskActionClient()
|
||||
.submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval()));
|
||||
|
||||
// Verify none of these segments have versions > lock version
|
||||
for (final DataSegment unusedSegment : unusedSegments) {
|
||||
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
|
||||
throw new ISE(
|
||||
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
|
||||
unusedSegment.getIdentifier(),
|
||||
unusedSegment.getVersion(),
|
||||
myLock.getVersion()
|
||||
);
|
||||
}
|
||||
|
||||
log.info("OK to restore segment: %s", unusedSegment.getIdentifier());
|
||||
}
|
||||
|
||||
List<DataSegment> restoredSegments = Lists.newLinkedList();
|
||||
|
||||
// Move segments
|
||||
for (DataSegment segment : unusedSegments) {
|
||||
restoredSegments.add(toolbox.getDataSegmentArchiver().restore(segment));
|
||||
}
|
||||
|
||||
// Update metadata for moved segments
|
||||
toolbox.getTaskActionClient().submit(
|
||||
new SegmentMetadataUpdateAction(
|
||||
ImmutableSet.copyOf(restoredSegments)
|
||||
)
|
||||
);
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
}
|
|
@ -47,6 +47,7 @@ import io.druid.query.QueryRunner;
|
|||
@JsonSubTypes.Type(name = "kill", value = KillTask.class),
|
||||
@JsonSubTypes.Type(name = "move", value = MoveTask.class),
|
||||
@JsonSubTypes.Type(name = "archive", value = ArchiveTask.class),
|
||||
@JsonSubTypes.Type(name = "restore", value = RestoreTask.class),
|
||||
@JsonSubTypes.Type(name = "index", value = IndexTask.class),
|
||||
@JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class),
|
||||
@JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class),
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.google.common.collect.Ordering;
|
|||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.db.DbConnector;
|
||||
import io.druid.db.DbTablesConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.TimelineObjectHolder;
|
||||
|
@ -89,7 +90,7 @@ public class IndexerDBCoordinator
|
|||
final ResultIterator<Map<String, Object>> dbSegments =
|
||||
handle.createQuery(
|
||||
String.format(
|
||||
"SELECT payload FROM %s WHERE used = 1 AND dataSource = :dataSource",
|
||||
"SELECT payload FROM %s WHERE used = true AND dataSource = :dataSource",
|
||||
dbTables.getSegmentsTable()
|
||||
)
|
||||
)
|
||||
|
@ -179,6 +180,9 @@ public class IndexerDBCoordinator
|
|||
try {
|
||||
handle.createStatement(
|
||||
String.format(
|
||||
DbConnector.isPostgreSQL(handle) ?
|
||||
"INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) "
|
||||
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)":
|
||||
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) "
|
||||
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
|
||||
dbTables.getSegmentsTable()
|
||||
|
@ -196,6 +200,8 @@ public class IndexerDBCoordinator
|
|||
.execute();
|
||||
|
||||
log.info("Published segment [%s] to DB", segment.getIdentifier());
|
||||
} catch(SQLException e) {
|
||||
throw new IOException(e);
|
||||
} catch(Exception e) {
|
||||
if (e.getCause() instanceof SQLException && segmentExists(handle, segment)) {
|
||||
log.info("Found [%s] in DB, not updating DB", segment.getIdentifier());
|
||||
|
@ -293,11 +299,13 @@ public class IndexerDBCoordinator
|
|||
new HandleCallback<List<DataSegment>>()
|
||||
{
|
||||
@Override
|
||||
public List<DataSegment> withHandle(Handle handle) throws IOException
|
||||
public List<DataSegment> withHandle(Handle handle) throws IOException, SQLException
|
||||
{
|
||||
return handle.createQuery(
|
||||
String.format(
|
||||
"SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = 0",
|
||||
DbConnector.isPostgreSQL(handle)?
|
||||
"SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and \"end\" <= :end and used = false":
|
||||
"SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = false",
|
||||
dbTables.getSegmentsTable()
|
||||
)
|
||||
)
|
||||
|
|
|
@ -38,6 +38,7 @@ import io.druid.indexing.common.TaskToolboxFactory;
|
|||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.query.NoopQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QuerySegmentWalker;
|
||||
import io.druid.query.SegmentDescriptor;
|
||||
|
@ -152,10 +153,17 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
|||
private <T> QueryRunner<T> getQueryRunnerImpl(Query<T> query)
|
||||
{
|
||||
QueryRunner<T> queryRunner = null;
|
||||
String queryDataSource;
|
||||
try {
|
||||
queryDataSource = ((TableDataSource)query.getDataSource()).getName();
|
||||
}
|
||||
catch (ClassCastException e) {
|
||||
throw new IllegalArgumentException("Subqueries are not welcome here");
|
||||
}
|
||||
|
||||
for (final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem : ImmutableList.copyOf(runningItems)) {
|
||||
final Task task = taskRunnerWorkItem.getTask();
|
||||
if (task.getDataSource().equals(query.getDataSource())) {
|
||||
if (task.getDataSource().equals(queryDataSource)) {
|
||||
final QueryRunner<T> taskQueryRunner = task.getQueryRunner(query);
|
||||
|
||||
if (taskQueryRunner != null) {
|
||||
|
@ -163,7 +171,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
|||
queryRunner = taskQueryRunner;
|
||||
} else {
|
||||
log.makeAlert("Found too many query runners for datasource")
|
||||
.addData("dataSource", query.getDataSource())
|
||||
.addData("dataSource", queryDataSource)
|
||||
.emit();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,7 +46,6 @@ import io.druid.indexing.overlord.scaling.ResourceManagementScheduler;
|
|||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import io.druid.tasklogs.TaskLogStreamer;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.ws.rs.Consumes;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue