Merge branch 'master' into new-schema

Conflicts:
	indexing-hadoop/src/main/java/io/druid/indexer/DbUpdaterJob.java
	indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java
	indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java
	server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java
	server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java
This commit is contained in:
fjy 2014-04-25 14:03:28 -07:00
commit 76e0a48527
291 changed files with 11439 additions and 1990 deletions

Binary file not shown.

Binary file not shown.

View File

@ -30,4 +30,4 @@ echo "For examples, see: "
echo " " echo " "
ls -1 examples/*/*sh ls -1 examples/*/*sh
echo " " echo " "
echo "See also http://druid.io/docs/0.6.69" echo "See also http://druid.io/docs/latest"

View File

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

View File

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

View File

@ -23,7 +23,6 @@ import com.google.common.base.Supplier;
import com.google.common.base.Throwables; import com.google.common.base.Throwables;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.lifecycle.LifecycleStop;
@ -31,16 +30,8 @@ import com.metamx.common.logger.Logger;
import io.druid.db.DbConnector; import io.druid.db.DbConnector;
import io.druid.db.DbTablesConfig; import io.druid.db.DbTablesConfig;
import org.joda.time.Duration; import org.joda.time.Duration;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.tweak.ResultSetMapper;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.Arrays; import java.util.Arrays;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
@ -57,38 +48,25 @@ public class ConfigManager
private final Object lock = new Object(); private final Object lock = new Object();
private boolean started = false; private boolean started = false;
private final IDBI dbi; private final DbConnector dbConnector;
private final Supplier<ConfigManagerConfig> config; private final Supplier<ConfigManagerConfig> config;
private final ScheduledExecutorService exec; private final ScheduledExecutorService exec;
private final ConcurrentMap<String, ConfigHolder> watchedConfigs; private final ConcurrentMap<String, ConfigHolder> watchedConfigs;
private final String selectStatement; private final String configTable;
private final String insertStatement;
private volatile ConfigManager.PollingCallable poller; private volatile ConfigManager.PollingCallable poller;
@Inject @Inject
public ConfigManager(IDBI dbi, Supplier<DbTablesConfig> dbTables, Supplier<ConfigManagerConfig> config) public ConfigManager(DbConnector dbConnector, Supplier<DbTablesConfig> dbTables, Supplier<ConfigManagerConfig> config)
{ {
this.dbi = dbi; this.dbConnector = dbConnector;
this.config = config; this.config = config;
this.exec = ScheduledExecutors.fixed(1, "config-manager-%s"); this.exec = ScheduledExecutors.fixed(1, "config-manager-%s");
this.watchedConfigs = Maps.newConcurrentMap(); this.watchedConfigs = Maps.newConcurrentMap();
final String configTable = dbTables.get().getConfigTable(); this.configTable = dbTables.get().getConfigTable();
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
);
} }
@LifecycleStart @LifecycleStart
@ -127,7 +105,7 @@ public class ConfigManager
{ {
for (Map.Entry<String, ConfigHolder> entry : watchedConfigs.entrySet()) { for (Map.Entry<String, ConfigHolder> entry : watchedConfigs.entrySet()) {
try { try {
if (entry.getValue().swapIfNew(lookup(entry.getKey()))) { if (entry.getValue().swapIfNew(dbConnector.lookup(configTable, "name", "payload", entry.getKey()))) {
log.info("New value for key[%s] seen.", entry.getKey()); log.info("New value for key[%s] seen.", entry.getKey());
} }
} }
@ -159,7 +137,7 @@ public class ConfigManager
// Multiple of these callables can be submitted at the same time, but the callables themselves // Multiple of these callables can be submitted at the same time, but the callables themselves
// are executed serially, so double check that it hasn't already been populated. // are executed serially, so double check that it hasn't already been populated.
if (!watchedConfigs.containsKey(key)) { if (!watchedConfigs.containsKey(key)) {
byte[] value = lookup(key); byte[] value = dbConnector.lookup(configTable, "name", "payload", key);
ConfigHolder<T> holder = new ConfigHolder<T>(value, serde); ConfigHolder<T> holder = new ConfigHolder<T>(value, serde);
watchedConfigs.put(key, holder); watchedConfigs.put(key, holder);
} }
@ -187,45 +165,10 @@ public class ConfigManager
return holder.getReference(); return holder.getReference();
} }
public byte[] lookup(final String key)
{
return dbi.withHandle(
new HandleCallback<byte[]>()
{
@Override
public byte[] withHandle(Handle handle) throws Exception
{
List<byte[]> matched = handle.createQuery(selectStatement)
.bind("name", key)
.map(
new ResultSetMapper<byte[]>()
{
@Override
public byte[] map(int index, ResultSet r, StatementContext ctx)
throws SQLException
{
return r.getBytes("payload");
}
}
).list();
if (matched.isEmpty()) {
return null;
}
if (matched.size() > 1) {
throw new ISE("Error! More than one matching entry[%d] found for [%s]?!", matched.size(), key);
}
return matched.get(0);
}
}
);
}
public <T> boolean set(final String key, final ConfigSerde<T> serde, final T obj) public <T> boolean set(final String key, final ConfigSerde<T> serde, final T obj)
{ {
if (obj == null) { if (obj == null || !started) {
return false; return false;
} }
@ -238,20 +181,7 @@ public class ConfigManager
@Override @Override
public Boolean call() throws Exception public Boolean call() throws Exception
{ {
dbi.withHandle( dbConnector.insertOrUpdate(configTable, "name", "payload", key, newBytes);
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(insertStatement)
.bind("name", key)
.bind("payload", newBytes)
.execute();
return null;
}
}
);
final ConfigHolder configHolder = watchedConfigs.get(key); final ConfigHolder configHolder = watchedConfigs.get(key);
if (configHolder != null) { if (configHolder != null) {

View File

@ -21,14 +21,18 @@ package io.druid.db;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import org.apache.commons.dbcp.BasicDataSource; import org.apache.commons.dbcp.BasicDataSource;
import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.tweak.ResultSetMapper;
import javax.sql.DataSource; import javax.sql.DataSource;
import java.sql.ResultSet;
import java.sql.SQLException; import java.sql.SQLException;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -39,58 +43,61 @@ public class DbConnector
{ {
private static final Logger log = new Logger(DbConnector.class); private static final Logger log = new Logger(DbConnector.class);
public static void createSegmentTable(final IDBI dbi, final String segmentTableName) public static void createSegmentTable(final IDBI dbi, final String segmentTableName, boolean isPostgreSQL)
{ {
createTable( createTable(
dbi, dbi,
segmentTableName, segmentTableName,
String.format( String.format(
isPostgreSQL(dbi) ? isPostgreSQL ?
"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 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(dataSource);"+
"CREATE INDEX ON %1$s(used);": "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))", "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 segmentTableName
) ),
isPostgreSQL
); );
} }
public static void createRuleTable(final IDBI dbi, final String ruleTableName) public static void createRuleTable(final IDBI dbi, final String ruleTableName, boolean isPostgreSQL)
{ {
createTable( createTable(
dbi, dbi,
ruleTableName, ruleTableName,
String.format( String.format(
isPostgreSQL(dbi) ? isPostgreSQL ?
"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 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 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))", "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 ruleTableName
) ),
isPostgreSQL
); );
} }
public static void createConfigTable(final IDBI dbi, final String configTableName) public static void createConfigTable(final IDBI dbi, final String configTableName, boolean isPostgreSQL)
{ {
createTable( createTable(
dbi, dbi,
configTableName, configTableName,
String.format( String.format(
isPostgreSQL(dbi) ? isPostgreSQL ?
"CREATE TABLE %s (name VARCHAR(255) NOT NULL, payload bytea NOT NULL, PRIMARY KEY(name))": "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))", "CREATE table %s (name VARCHAR(255) NOT NULL, payload BLOB NOT NULL, PRIMARY KEY(name))",
configTableName configTableName
) ),
isPostgreSQL
); );
} }
public static void createTaskTable(final IDBI dbi, final String taskTableName) public static void createTaskTable(final IDBI dbi, final String taskTableName, boolean isPostgreSQL)
{ {
createTable( createTable(
dbi, dbi,
taskTableName, taskTableName,
String.format( String.format(
isPostgreSQL(dbi) ? isPostgreSQL ?
"CREATE TABLE %1$s (\n" "CREATE TABLE %1$s (\n"
+ " id varchar(255) NOT NULL,\n" + " id varchar(255) NOT NULL,\n"
+ " created_date TEXT NOT NULL,\n" + " created_date TEXT NOT NULL,\n"
@ -112,17 +119,18 @@ public class DbConnector
+ " KEY (active, created_date(100))\n" + " KEY (active, created_date(100))\n"
+ ")", + ")",
taskTableName taskTableName
) ),
isPostgreSQL
); );
} }
public static void createTaskLogTable(final IDBI dbi, final String taskLogsTableName) public static void createTaskLogTable(final IDBI dbi, final String taskLogsTableName, boolean isPostgreSQL)
{ {
createTable( createTable(
dbi, dbi,
taskLogsTableName, taskLogsTableName,
String.format( String.format(
isPostgreSQL(dbi) ? isPostgreSQL ?
"CREATE TABLE %1$s (\n" "CREATE TABLE %1$s (\n"
+ " id bigserial NOT NULL,\n" + " id bigserial NOT NULL,\n"
+ " task_id varchar(255) DEFAULT NULL,\n" + " task_id varchar(255) DEFAULT NULL,\n"
@ -138,17 +146,18 @@ public class DbConnector
+ " KEY `task_id` (`task_id`)\n" + " KEY `task_id` (`task_id`)\n"
+ ")", + ")",
taskLogsTableName taskLogsTableName
) ),
isPostgreSQL
); );
} }
public static void createTaskLockTable(final IDBI dbi, final String taskLocksTableName) public static void createTaskLockTable(final IDBI dbi, final String taskLocksTableName, boolean isPostgreSQL)
{ {
createTable( createTable(
dbi, dbi,
taskLocksTableName, taskLocksTableName,
String.format( String.format(
isPostgreSQL(dbi) ? isPostgreSQL ?
"CREATE TABLE %1$s (\n" "CREATE TABLE %1$s (\n"
+ " id bigserial NOT NULL,\n" + " id bigserial NOT NULL,\n"
+ " task_id varchar(255) DEFAULT NULL,\n" + " task_id varchar(255) DEFAULT NULL,\n"
@ -164,14 +173,16 @@ public class DbConnector
+ " KEY `task_id` (`task_id`)\n" + " KEY `task_id` (`task_id`)\n"
+ ")", + ")",
taskLocksTableName taskLocksTableName
) ),
isPostgreSQL
); );
} }
public static void createTable( public static void createTable(
final IDBI dbi, final IDBI dbi,
final String tableName, final String tableName,
final String sql final String sql,
final boolean isPostgreSQL
) )
{ {
try { try {
@ -182,7 +193,7 @@ public class DbConnector
public Void withHandle(Handle handle) throws Exception public Void withHandle(Handle handle) throws Exception
{ {
List<Map<String, Object>> table; List<Map<String, Object>> table;
if ( isPostgreSQL(dbi) ) { if ( isPostgreSQL ) {
table = handle.select(String.format("SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = 'public' AND tablename LIKE '%s'", tableName)); table = handle.select(String.format("SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = 'public' AND tablename LIKE '%s'", tableName));
} else { } else {
table = handle.select(String.format("SHOW tables LIKE '%s'", tableName)); table = handle.select(String.format("SHOW tables LIKE '%s'", tableName));
@ -205,6 +216,84 @@ public class DbConnector
} }
} }
public Void insertOrUpdate(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key,
final byte[] value
) throws SQLException
{
final String insertOrUpdateStatement = String.format(
isPostgreSQL ?
"BEGIN;\n" +
"LOCK TABLE %1$s IN SHARE ROW EXCLUSIVE MODE;\n" +
"WITH upsert AS (UPDATE %1$s SET %3$s=:value WHERE %2$s=:key RETURNING *)\n" +
" INSERT INTO %1$s (%2$s, %3$s) SELECT :key, :value WHERE NOT EXISTS (SELECT * FROM upsert)\n;" +
"COMMIT;" :
"INSERT INTO %1$s (%2$s, %3$s) VALUES (:key, :value) ON DUPLICATE KEY UPDATE %3$s = :value",
tableName, keyColumn, valueColumn
);
return dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(insertOrUpdateStatement)
.bind("key", key)
.bind("value", value)
.execute();
return null;
}
}
);
}
public byte[] lookup(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key
)
{
final String selectStatement = String.format("SELECT %s FROM %s WHERE %s = :key", valueColumn, tableName, keyColumn);
return dbi.withHandle(
new HandleCallback<byte[]>()
{
@Override
public byte[] withHandle(Handle handle) throws Exception
{
List<byte[]> matched = handle.createQuery(selectStatement)
.bind("key", key)
.map(
new ResultSetMapper<byte[]>()
{
@Override
public byte[] map(int index, ResultSet r, StatementContext ctx)
throws SQLException
{
return r.getBytes(valueColumn);
}
}
).list();
if (matched.isEmpty()) {
return null;
}
if (matched.size() > 1) {
throw new ISE("Error! More than one matching entry[%d] found for [%s]?!", matched.size(), key);
}
return matched.get(0);
}
}
);
}
public static Boolean isPostgreSQL(final IDBI dbi) public static Boolean isPostgreSQL(final IDBI dbi)
{ {
return dbi.withHandle( return dbi.withHandle(
@ -219,7 +308,7 @@ public class DbConnector
); );
} }
public static Boolean isPostgreSQL(final Handle handle) throws SQLException protected static Boolean isPostgreSQL(final Handle handle) throws SQLException
{ {
return handle.getConnection().getMetaData().getDatabaseProductName().contains("PostgreSQL"); return handle.getConnection().getMetaData().getDatabaseProductName().contains("PostgreSQL");
} }
@ -227,6 +316,7 @@ public class DbConnector
private final Supplier<DbConnectorConfig> config; private final Supplier<DbConnectorConfig> config;
private final Supplier<DbTablesConfig> dbTables; private final Supplier<DbTablesConfig> dbTables;
private final DBI dbi; private final DBI dbi;
private boolean isPostgreSQL = false;
@Inject @Inject
public DbConnector(Supplier<DbConnectorConfig> config, Supplier<DbTablesConfig> dbTables) public DbConnector(Supplier<DbConnectorConfig> config, Supplier<DbTablesConfig> dbTables)
@ -242,6 +332,11 @@ public class DbConnector
return dbi; return dbi;
} }
public boolean isPostgreSQL()
{
return isPostgreSQL;
}
private DataSource getDatasource() private DataSource getDatasource()
{ {
DbConnectorConfig connectorConfig = config.get(); DbConnectorConfig connectorConfig = config.get();
@ -249,7 +344,9 @@ public class DbConnector
BasicDataSource dataSource = new BasicDataSource(); BasicDataSource dataSource = new BasicDataSource();
dataSource.setUsername(connectorConfig.getUser()); dataSource.setUsername(connectorConfig.getUser());
dataSource.setPassword(connectorConfig.getPassword()); dataSource.setPassword(connectorConfig.getPassword());
dataSource.setUrl(connectorConfig.getConnectURI()); String uri = connectorConfig.getConnectURI();
isPostgreSQL = uri.startsWith("jdbc:postgresql");
dataSource.setUrl(uri);
if (connectorConfig.isUseValidationQuery()) { if (connectorConfig.isUseValidationQuery()) {
dataSource.setValidationQuery(connectorConfig.getValidationQuery()); dataSource.setValidationQuery(connectorConfig.getValidationQuery());
@ -262,21 +359,21 @@ public class DbConnector
public void createSegmentTable() public void createSegmentTable()
{ {
if (config.get().isCreateTables()) { if (config.get().isCreateTables()) {
createSegmentTable(dbi, dbTables.get().getSegmentsTable()); createSegmentTable(dbi, dbTables.get().getSegmentsTable(), isPostgreSQL);
} }
} }
public void createRulesTable() public void createRulesTable()
{ {
if (config.get().isCreateTables()) { if (config.get().isCreateTables()) {
createRuleTable(dbi, dbTables.get().getRulesTable()); createRuleTable(dbi, dbTables.get().getRulesTable(), isPostgreSQL);
} }
} }
public void createConfigTable() public void createConfigTable()
{ {
if (config.get().isCreateTables()) { if (config.get().isCreateTables()) {
createConfigTable(dbi, dbTables.get().getConfigTable()); createConfigTable(dbi, dbTables.get().getConfigTable(), isPostgreSQL);
} }
} }
@ -284,9 +381,9 @@ public class DbConnector
{ {
if (config.get().isCreateTables()) { if (config.get().isCreateTables()) {
final DbTablesConfig dbTablesConfig = dbTables.get(); final DbTablesConfig dbTablesConfig = dbTables.get();
createTaskTable(dbi, dbTablesConfig.getTasksTable()); createTaskTable(dbi, dbTablesConfig.getTasksTable(), isPostgreSQL);
createTaskLogTable(dbi, dbTablesConfig.getTaskLogTable()); createTaskLogTable(dbi, dbTablesConfig.getTaskLogTable(), isPostgreSQL);
createTaskLockTable(dbi, dbTablesConfig.getTaskLockTable()); createTaskLockTable(dbi, dbTablesConfig.getTaskLockTable(), isPostgreSQL);
} }
} }
} }

View File

@ -66,6 +66,6 @@ public class JacksonConfigManagerModule implements Module
} }
); );
return new ConfigManager(dbConnector.getDBI(), dbTables, config); return new ConfigManager(dbConnector, dbTables, config);
} }
} }

View File

@ -54,10 +54,8 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim
"gran": "day" "gran": "day"
}, },
"pathSpec": { "pathSpec": {
"type": "granularity", "type": "static",
"dataGranularity": "hour", "paths" : "example/path/data.gz,example/path/moredata.gz"
"inputPath": "s3n:\/\/billy-bucket\/the\/data\/is\/here",
"filePattern": ".*"
}, },
"rollupSpec": { "rollupSpec": {
"aggs": [ "aggs": [
@ -116,6 +114,20 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim
There are multiple types of path specification: There are multiple types of path specification:
##### `static`
Is a type of data loader where a static path to where the data files are located is passed.
|property|description|required?|
|--------|-----------|---------|
|paths|A String of input paths indicating where the raw data is located.|yes|
For example, using the static input paths:
```
"paths" : "s3n://billy-bucket/the/data/is/here/data.gz, s3n://billy-bucket/the/data/is/here/moredata.gz, s3n://billy-bucket/the/data/is/here/evenmoredata.gz"
```
##### `granularity` ##### `granularity`
Is a type of data loader that expects data to be laid out in a specific path format. Specifically, it expects it to be segregated by day in this directory format `y=XXXX/m=XX/d=XX/H=XX/M=XX/S=XX` (dates are represented by lowercase, time is represented by uppercase). Is a type of data loader that expects data to be laid out in a specific path format. Specifically, it expects it to be segregated by day in this directory format `y=XXXX/m=XX/d=XX/H=XX/M=XX/S=XX` (dates are represented by lowercase, time is represented by uppercase).
@ -155,9 +167,20 @@ For example, data for a day may be split by the dimension "last\_name" into two
In hashed partition type, the number of partitions is determined based on the targetPartitionSize and cardinality of input set and the data is partitioned based on the hashcode of the row. 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. It is recommended to use Hashed partition as it is more efficient than singleDimension since it does not need to determine the dimension for creating partitions.
Hashing also gives better distribution of data resulting in equal sized partitons and improving query performance Hashing also gives better distribution of data resulting in equal sized partitions and improving query performance
To use this option, the indexer must be given a target partition size. It can then find a good set of partition ranges on its own. To use this druid to automatically determine optimal partitions indexer must be given a target partition size. It can then find a good set of partition ranges on its own.
#### Configuration for disabling auto-sharding and creating Fixed number of partitions
Druid can be configured to NOT run determine partitions and create a fixed number of shards by specifying numShards in hashed partitionsSpec.
e.g This configuration will skip determining optimal partitions and always create 4 shards for every segment granular interval
```json
"partitionsSpec": {
"type": "hashed"
"numShards": 4
}
```
|property|description|required?| |property|description|required?|
|--------|-----------|---------| |--------|-----------|---------|
@ -165,6 +188,7 @@ To use this option, the indexer must be given a target partition size. It can th
|targetPartitionSize|target number of rows to include in a partition, should be a number that targets segments of 700MB\~1GB.|yes| |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| |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| |assumeGrouped|assume input data has already been grouped on time and dimensions. This is faster, but can choose suboptimal partitions if the assumption is violated.|no|
|numShards|provides a way to manually override druid-auto sharding and specify the number of shards to create for each segment granular interval.It is only supported by hashed partitionSpec and targetPartitionSize must be set to -1|no|
### Updater job spec ### Updater job spec
@ -215,7 +239,7 @@ The schema of the Hadoop Index Task contains a task "type" and a Hadoop Index Co
"type" : "static", "type" : "static",
"paths" : "data.json" "paths" : "data.json"
}, },
"targetPartitionSi:qze" : 5000000, "targetPartitionSize" : 5000000,
"rollupSpec" : { "rollupSpec" : {
"aggs": [{ "aggs": [{
"type" : "count", "type" : "count",

View File

@ -81,7 +81,7 @@ druid.server.http.numThreads=50
druid.request.logging.type=emitter druid.request.logging.type=emitter
druid.request.logging.feed=druid_requests druid.request.logging.feed=druid_requests
druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor", "io.druid.client.cache.CacheMonitor"] druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]
# Emit metrics over http # Emit metrics over http
druid.emitter=http druid.emitter=http
@ -106,16 +106,16 @@ The broker module uses several of the default modules in [Configuration](Configu
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.broker.cache.sizeInBytes`|Maximum size of the cache. If this is zero, cache is disabled.|0| |`druid.broker.cache.sizeInBytes`|Maximum cache size in bytes. Zero disables caching.|0|
|`druid.broker.cache.initialSize`|The initial size of the cache in bytes.|500000| |`druid.broker.cache.initialSize`|Initial size of the hashtable backing the cache.|500000|
|`druid.broker.cache.logEvictionCount`|If this is non-zero, there will be an eviction of entries.|0| |`druid.broker.cache.logEvictionCount`|If non-zero, log cache eviction every `logEvictionCount` items.|0|
#### Memcache #### Memcache
|Property|Description|Default| |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.expiration`|Memcached [expiration time](https://code.google.com/p/memcached/wiki/NewCommands#Standard_Protocol).|2592000 (30 days)|
|`druid.broker.cache.timeout`|Maximum time in milliseconds to wait for a response from Memcache.|500| |`druid.broker.cache.timeout`|Maximum time in milliseconds to wait for a response from Memcached.|500|
|`druid.broker.cache.hosts`|Memcache hosts.|none| |`druid.broker.cache.hosts`|Command separated list of Memcached hosts `<host:port>`.|none|
|`druid.broker.cache.maxObjectSize`|Maximum object size in bytes for a Memcache object.|52428800 (50 MB)| |`druid.broker.cache.maxObjectSize`|Maximum object size in bytes for a Memcached object.|52428800 (50 MB)|
|`druid.broker.cache.memcachedPrefix`|Key prefix for all keys in Memcache.|druid| |`druid.broker.cache.memcachedPrefix`|Key prefix for all keys in Memcached.|druid|

View File

@ -4,7 +4,7 @@ layout: doc_page
# Configuring Druid # Configuring Druid
This describes the basic server configuration that is loaded by all the server processes; the same file is loaded by all. See also the json "specFile" descriptions in [Realtime](Realtime.html) and [Batch-ingestion](Batch-ingestion.html). This describes the basic server configuration that is loaded by all Druid server processes; the same file is loaded by all. See also the JSON "specFile" descriptions in [Realtime](Realtime.html) and [Batch-ingestion](Batch-ingestion.html).
## JVM Configuration Best Practices ## JVM Configuration Best Practices
@ -26,7 +26,7 @@ Note: as a future item, wed like to consolidate all of the various configurat
### Emitter Module ### Emitter Module
The Druid servers emit various metrics and alerts via something we call an Emitter. There are two emitter implementations included with the code, one that just logs to log4j and one that does POSTs of JSON events to a server. The properties for using the logging emitter are described below. The Druid servers emit various metrics and alerts via something we call an Emitter. There are two emitter implementations included with the code, one that just logs to log4j ("logging", which is used by default if no emitter is specified) and one that does POSTs of JSON events to a server ("http"). The properties for using the logging emitter are described below.
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
@ -47,7 +47,7 @@ The Druid servers emit various metrics and alerts via something we call an Emitt
|`druid.emitter.http.timeOut`|The timeout for data reads.|PT5M| |`druid.emitter.http.timeOut`|The timeout for data reads.|PT5M|
|`druid.emitter.http.flushMillis`|How often to internal message buffer is flushed (data is sent).|60000| |`druid.emitter.http.flushMillis`|How often to internal message buffer is flushed (data is sent).|60000|
|`druid.emitter.http.flushCount`|How many messages can the internal message buffer hold before flushing (sending).|500| |`druid.emitter.http.flushCount`|How many messages can the internal message buffer hold before flushing (sending).|500|
|`druid.emitter.http.recipientBaseUrl`|The base URL to emit messages to.|none| |`druid.emitter.http.recipientBaseUrl`|The base URL to emit messages to. Druid will POST JSON to be consumed at the HTTP endpoint specified by this property.|none|
### Http Client Module ### Http Client Module
@ -56,7 +56,7 @@ This is the HTTP client used by [Broker](Broker.html) nodes.
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to historical and real-time nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|5| |`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to historical and real-time nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|5|
|`druid.broker.http.readTimeout`|The timeout for data reads.|none| |`druid.broker.http.readTimeout`|The timeout for data reads.|PT15M|
### Curator Module ### Curator Module
@ -64,17 +64,17 @@ Druid uses [Curator](http://curator.incubator.apache.org/) for all [Zookeeper](h
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.zk.service.host`|The Zookeeper hosts to connect to.|none| |`druid.zk.service.host`|The ZooKeeper hosts to connect to. This is a REQUIRED property and therefore a host address must be supplied.|none|
|`druid.zk.service.sessionTimeoutMs`|Zookeeper session timeout.|30000| |`druid.zk.service.sessionTimeoutMs`|ZooKeeper session timeout, in milliseconds.|30000|
|`druid.curator.compress`|Boolean flag for whether or not created Znodes should be compressed.|false| |`druid.curator.compress`|Boolean flag for whether or not created Znodes should be compressed.|false|
### Announcer Module ### Announcer Module
The announcer module is used to announce and unannounce Znodes in Zookeeper (using Curator). The announcer module is used to announce and unannounce Znodes in ZooKeeper (using Curator).
#### Zookeeper Paths #### ZooKeeper Paths
See [Zookeeper](Zookeeper.html). See [ZooKeeper](ZooKeeper.html).
#### Data Segment Announcer #### Data Segment Announcer
@ -84,11 +84,11 @@ Data segment announcers are used to announce segments.
|--------|-----------|-------| |--------|-----------|-------|
|`druid.announcer.type`|Choices: legacy or batch. The type of data segment announcer to use.|legacy| |`druid.announcer.type`|Choices: legacy or batch. The type of data segment announcer to use.|legacy|
#### Single Data Segment Announcer ##### Single Data Segment Announcer
In legacy Druid, each segment served by a node would be announced as an individual Znode. In legacy Druid, each segment served by a node would be announced as an individual Znode.
#### Batch Data Segment Announcer ##### Batch Data Segment Announcer
In current Druid, multiple data segments may be announced under the same Znode. In current Druid, multiple data segments may be announced under the same Znode.
@ -105,16 +105,8 @@ This module contains query processing functionality.
|--------|-----------|-------| |--------|-----------|-------|
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| |`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)|
|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s| |`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, this means that even under heavy load there will still be one core available to do background tasks like talking with ZK and pulling down segments.|1| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
### AWS Module
This module is used to interact with S3.
|Property|Description|Default|
|--------|-----------|-------|
|`druid.s3.accessKey`|The access key to use to access S3.|none|
|`druid.s3.secretKey`|The secret key to use to access S3.|none|
### Metrics Module ### Metrics Module
@ -123,7 +115,15 @@ The metrics module is used to track Druid metrics.
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.monitoring.emissionPeriod`|How often metrics are emitted.|PT1m| |`druid.monitoring.emissionPeriod`|How often metrics are emitted.|PT1m|
|`druid.monitoring.monitors`|List of Druid monitors.|none| |`druid.monitoring.monitors`|Sets list of Druid monitors used by a node. Each monitor is specified as `com.metamx.metrics.<monitor-name>` (see below for names and more information). For example, you can specify monitors for a Broker with `druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]`.|none (no monitors)|
The following monitors are available:
* CacheMonitor &ndash; Emits metrics (to logs) about the segment results cache for Historical and Broker nodes. Reports typical cache statistics include hits, misses, rates, and size (bytes and number of entries), as well as timeouts and and errors.
* SysMonitor &ndash; This uses the [SIGAR library](http://www.hyperic.com/products/sigar) to report on various system activities and statuses.
* ServerMonitor &ndash; Reports statistics on Historical nodes.
* JvmMonitor &ndash; Reports JVM-related statistics.
* RealtimeMetricsMonitor &ndash; Reports statistics on Realtime nodes.
### Server Module ### Server Module
@ -137,22 +137,24 @@ This module is used for Druid server nodes.
### Storage Node Module ### Storage Node Module
This module is used by nodes that store data (historical and real-time nodes). This module is used by nodes that store data (Historical and Realtime).
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.server.maxSize`|The maximum number of bytes worth of segments that the node wants assigned to it. This is not a limit that the historical nodes actually enforce, they just publish it to the coordinator and trust the coordinator to do the right thing|0| |`druid.server.maxSize`|The maximum number of bytes-worth of segments that the node wants assigned to it. This is not a limit that Historical nodes actually enforce, just a value published to the Coordinator node so it can plan accordingly.|0|
|`druid.server.tier`|Druid server host port.|none| |`druid.server.tier`| A string to name the distribution tier that the storage node belongs to. Many of the [rules Coordinator nodes use](Rule-Configuration.html) to manage segments can be keyed on tiers. | `_default_tier` |
|`druid.server.priority`|In a tiered architecture, the priority of the tier, thus allowing control over which nodes are queried. Higher numbers mean higher priority. The default (no priority) works for architecture with no cross replication (tiers that have no data-storage overlap). Data centers typically have equal priority. | 0 |
#### Segment Cache #### Segment Cache
Druid storage nodes maintain information about segments they have already downloaded. Druid storage nodes maintain information about segments they have already downloaded, and a disk cache to store that data.
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.segmentCache.locations`|Segments assigned to a historical node are first stored on the local file system and then served by the historical node. These locations define where that local cache resides|none| |`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. | none (no caching) |
|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true| |`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true|
|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| |`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir|
### Jetty Server Module ### Jetty Server Module
@ -193,7 +195,7 @@ This module is required by nodes that can serve queries.
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.query.chunkPeriod`|Long interval queries may be broken into shorter interval queries.|P1M| |`druid.query.chunkPeriod`|Long-interval queries (of any type) may be broken into shorter interval queries, reducing the impact on resources. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. |0 (off)|
#### GroupBy Query Config #### GroupBy Query Config
@ -210,17 +212,28 @@ This module is required by nodes that can serve queries.
|--------|-----------|-------| |--------|-----------|-------|
|`druid.query.search.maxSearchLimit`|Maximum number of search results to return.|1000| |`druid.query.search.maxSearchLimit`|Maximum number of search results to return.|1000|
### Discovery Module ### Discovery Module
The discovery module is used for service discovery. The discovery module is used for service discovery.
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.discovery.curator.path`|Services announce themselves under this Zookeeper path.|/druid/discovery| |`druid.discovery.curator.path`|Services announce themselves under this ZooKeeper path.|/druid/discovery|
#### Indexing Service Discovery Module
This module is used to find the [Indexing Service](Indexing-Service.html) using Curator service discovery.
|Property|Description|Default|
|--------|-----------|-------|
|`druid.selectors.indexing.serviceName`|The druid.service name of the indexing service Overlord node. To start the Overlord with a different name, set it with this property. |overlord|
### Server Inventory View Module ### Server Inventory View Module
This module is used to read announcements of segments in Zookeeper. The configs are identical to the Announcer Module. This module is used to read announcements of segments in ZooKeeper. The configs are identical to the Announcer Module.
### Database Connector Module ### Database Connector Module
@ -228,7 +241,6 @@ These properties specify the jdbc connection and other configuration around the
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.db.connector.pollDuration`|The jdbc connection URI.|none|
|`druid.db.connector.user`|The username to connect with.|none| |`druid.db.connector.user`|The username to connect with.|none|
|`druid.db.connector.password`|The password to connect with.|none| |`druid.db.connector.password`|The password to connect with.|none|
|`druid.db.connector.createTables`|If Druid requires a table and it doesn't exist, create it?|true| |`druid.db.connector.createTables`|If Druid requires a table and it doesn't exist, create it?|true|
@ -250,13 +262,6 @@ The Jackson Config manager reads and writes config entries from the Druid config
|--------|-----------|-------| |--------|-----------|-------|
|`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1m| |`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1m|
### Indexing Service Discovery Module
This module is used to find the [Indexing Service](Indexing-Service.html) using Curator service discovery.
|Property|Description|Default|
|--------|-----------|-------|
|`druid.selectors.indexing.serviceName`|The druid.service name of the indexing service Overlord node.|none|
### DataSegment Pusher/Puller Module ### DataSegment Pusher/Puller Module
@ -290,6 +295,16 @@ This deep storage is used to interface with Amazon's S3.
|`druid.storage.archiveBucket`|S3 bucket name for archiving when running the indexing-service *archive task*.|none| |`druid.storage.archiveBucket`|S3 bucket name for archiving when running the indexing-service *archive task*.|none|
|`druid.storage.archiveBaseKey`|S3 object key prefix for archiving.|none| |`druid.storage.archiveBaseKey`|S3 object key prefix for archiving.|none|
#### AWS Module
This module is used to interact with S3.
|Property|Description|Default|
|--------|-----------|-------|
|`druid.s3.accessKey`|The access key to use to access S3.|none|
|`druid.s3.secretKey`|The secret key to use to access S3.|none|
#### HDFS Deep Storage #### HDFS Deep Storage
This deep storage is used to interface with HDFS. This deep storage is used to interface with HDFS.

View File

@ -20,7 +20,7 @@ io.druid.cli.Main server coordinator
Rules Rules
----- -----
Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different historical node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The coordinator loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different historical node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The coordinator loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule.
For more information on rules, see [Rule Configuration](Rule-Configuration.html). For more information on rules, see [Rule Configuration](Rule-Configuration.html).
@ -136,4 +136,4 @@ FAQ
No. If the Druid coordinator is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the coordinator node can be started up at any time, and after a configurable delay, will start running coordinator tasks. No. If the Druid coordinator is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the coordinator node can be started up at any time, and after a configurable delay, will start running coordinator tasks.
This also means that if you have a working cluster and all of your coordinators die, the cluster will continue to function, it just wont experience any changes to its data topology. This also means that if you have a working cluster and all of your coordinators die, the cluster will continue to function, it just wont experience any changes to its data topology.

View File

@ -19,13 +19,13 @@ Clone Druid and build it:
git clone https://github.com/metamx/druid.git druid git clone https://github.com/metamx/druid.git druid
cd druid cd druid
git fetch --tags git fetch --tags
git checkout druid-0.6.69 git checkout druid-0.6.101
./build.sh ./build.sh
``` ```
### Downloading the DSK (Druid Standalone Kit) ### Downloading the DSK (Druid Standalone Kit)
[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.69-bin.tar.gz) a stand-alone tarball and run it: [Download](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz) a stand-alone tarball and run it:
``` bash ``` bash
tar -xzf druid-services-0.X.X-bin.tar.gz tar -xzf druid-services-0.X.X-bin.tar.gz

View File

@ -2,13 +2,13 @@
layout: doc_page layout: doc_page
--- ---
# Aggregation Granularity # Aggregation Granularity
The granularity field determines how data gets bucketed across the time dimension, i.e how it gets aggregated by hour, day, minute, etc. 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. It can be specified either as a string for simple granularities or as an object for arbitrary granularities.
### Simple 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` Supported granularity strings are: `all`, `none`, `minute`, `fifteen_minute`, `thirty_minute`, `hour` and `day`
@ -35,25 +35,21 @@ This chunks up every hour on the half-hour.
### Period Granularities ### 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. Time zone is optional (defaults to UTC). Origin is optional (defaults to 1970-01-01T00:00:00 in the given time zone).
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)
``` ```
{"type": "period", "period": "P2D", "timeZone": "America/Los_Angeles"} {"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"} {"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).

View File

@ -2,7 +2,7 @@
layout: doc_page layout: doc_page
--- ---
# groupBy Queries # 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 aggreagates for some time range, we highly recommend using [TimeseriesQueries](TimeseriesQuery.html) instead. The performance will be substantially better. 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: An example groupBy query object is shown below:
``` json ``` json
@ -87,4 +87,4 @@ To pull it all together, the above query would return *n\*m* data points, up to
}, },
... ...
] ]
``` ```

View File

@ -36,8 +36,6 @@ druid.processing.numThreads=1
druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 10000000000}] 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 Production Configs
------------------ ------------------
These production configs are using S3 as a deep store. These production configs are using S3 as a deep store.

View File

@ -66,7 +66,7 @@ druid.host=#{IP_ADDR}:8080
druid.port=8080 druid.port=8080
druid.service=druid/prod/indexer druid.service=druid/prod/indexer
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.69"] druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101"]
druid.zk.service.host=#{ZK_IPs} druid.zk.service.host=#{ZK_IPs}
druid.zk.paths.base=/druid/prod druid.zk.paths.base=/druid/prod
@ -115,7 +115,7 @@ druid.host=#{IP_ADDR}:8080
druid.port=8080 druid.port=8080
druid.service=druid/prod/worker druid.service=druid/prod/worker
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.69","io.druid.extensions:druid-kafka-seven:0.6.69"] druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101","io.druid.extensions:druid-kafka-seven:0.6.101"]
druid.zk.service.host=#{ZK_IPs} druid.zk.service.host=#{ZK_IPs}
druid.zk.paths.base=/druid/prod druid.zk.paths.base=/druid/prod

View File

@ -39,9 +39,9 @@ Tasks are submitted to the overlord node in the form of JSON objects. Tasks can
``` ```
http://<OVERLORD_IP>:<port>/druid/indexer/v1/task http://<OVERLORD_IP>:<port>/druid/indexer/v1/task
``` ```
this will return you the taskId of the submitted task. this will return the taskId of the submitted task.
Tasks can cancelled via POST requests to: Tasks can be cancelled via POST requests to:
``` ```
http://<OVERLORD_IP>:<port>/druid/indexer/v1/task/{taskId}/shutdown http://<OVERLORD_IP>:<port>/druid/indexer/v1/task/{taskId}/shutdown

View File

@ -21,8 +21,12 @@ druid.storage.bucket=druid
druid.storage.baseKey=sample druid.storage.baseKey=sample
``` ```
## How do I get HDFS to work?
Make sure to include the `druid-hdfs-storage` module as one of your extensions and set `druid.storage.type=hdfs`.
## I don't see my Druid segments on my historical nodes ## 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"}] -Ddruid.segmentCache.locations=[{"path":"/tmp/druid/storageLocation","maxSize":"500000000000"}]
@ -31,7 +35,7 @@ You can check the coordinator console located at <COORDINATOR_IP>:<PORT>/cluster
## My queries are returning empty results ## 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>?interval=0/3000` 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. Note: the broker endpoint will only return valid results on historical segments.
## More information ## More information

View File

@ -3,18 +3,23 @@ layout: doc_page
--- ---
# Druid Plumbers # Druid Plumbers
The Plumber is the thing that handles generated segments both while they are being generated and when they are "done". This is also technically a pluggable interface and there are multiple implementations, but there are a lot of details handled by the plumber such that it is expected that there will only be a few implementations and only more advanced third-parties will implement their own. The plumber handles generated segments both while they are being generated and when they are "done". This is also technically a pluggable interface and there are multiple implementations. However, plumbers handle numerous complex details, and therefore an advanced understanding of Druid is recommended before implementing your own.
|Field|Type|Description|Required| |Field|Type|Description|Required|
|-----|----|-----------|--------| |-----|----|-----------|--------|
|type|String|Specifies the type of plumber. Each value will have its own configuration schema, plumbers packaged with Druid are described below.|yes| |type|String|Specifies the type of plumber. Each value will have its own configuration schema. Plumbers packaged with Druid are described below. The default type is "realtime".|yes|
We provide a brief description of the example to exemplify the types of things that are configured on the plumber. The following can be configured on the plumber:
* `windowPeriod` is the amount of lag time to allow events. This is configured with a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server.
* `basePersistDirectory` is the directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists.
* `maxPendingPersists` is how many persists a plumber can do concurrently without starting to block.
* `windowPeriod` is the amount of lag time to allow events. This is configured with a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server.
* `basePersistDirectory` is the directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists.
* `maxPendingPersists` is how many persists a plumber can do concurrently without starting to block.
* `segmentGranularity` specifies the granularity of the segment, or the amount of time a segment will represent.
* `rejectionPolicy` controls how data sets the data acceptance policy for creating and handing off segments. The following policies are available:
* `serverTime` &ndash; The recommended policy for "current time" data, it is optimal for current data that is generated and ingested in real time. Uses `windowPeriod` to accept only those events that are inside the window looking forward and back.
* `messageTime` &ndash; Can be used for non-"current time" as long as that data is relatively in sequence. Events are rejected if they are less than `windowPeriod` from the event with the latest timestamp. Hand off only occurs if an event is seen after the segmentGranularity and `windowPeriod`.
* `none` &ndash; Never hands off data unless shutdown() is called on the configured firehose.
* `test` &ndash; Useful for testing that handoff is working, *not useful in terms of data integrity*. It uses the sum of `segmentGranularity` plus `windowPeriod` as a window.
Available Plumbers Available Plumbers
------------------ ------------------

View File

@ -27,7 +27,7 @@ druid.host=localhost
druid.service=realtime druid.service=realtime
druid.port=8083 druid.port=8083
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.69"] druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.101"]
druid.zk.service.host=localhost druid.zk.service.host=localhost
@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080
druid.port=8080 druid.port=8080
druid.service=druid/prod/realtime druid.service=druid/prod/realtime
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.69","io.druid.extensions:druid-kafka-seven:0.6.69"] druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101","io.druid.extensions:druid-kafka-seven:0.6.101"]
druid.zk.service.host=#{ZK_IPs} druid.zk.service.host=#{ZK_IPs}
druid.zk.paths.base=/druid/prod druid.zk.paths.base=/druid/prod

View File

@ -1,6 +1,8 @@
--- ---
layout: doc_page layout: doc_page
--- ---
Realtime Data Ingestion Realtime Data Ingestion
======================= =======================
For general Real-time Node information, see [here](Realtime.html). For general Real-time Node information, see [here](Realtime.html).
@ -11,6 +13,7 @@ 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. Much of the configuration governing Realtime nodes and the ingestion of data is set in the Realtime spec file, discussed on this page.
<a id="realtime-specfile"></a> <a id="realtime-specfile"></a>
## Realtime "specFile" ## Realtime "specFile"
@ -81,6 +84,7 @@ This is a JSON Array so you can give more than one realtime stream to a given no
There are four parts to a realtime stream specification, `schema`, `config`, `firehose` and `plumber` which we will go into here. There are four parts to a realtime stream specification, `schema`, `config`, `firehose` and `plumber` which we will go into here.
### Schema ### Schema
This describes the data schema for the output Druid segment. More information about concepts in Druid and querying can be found at [Concepts-and-Terminology](Concepts-and-Terminology.html) and [Querying](Querying.html). This describes the data schema for the output Druid segment. More information about concepts in Druid and querying can be found at [Concepts-and-Terminology](Concepts-and-Terminology.html) and [Querying](Querying.html).
@ -92,6 +96,7 @@ This describes the data schema for the output Druid segment. More information ab
|indexGranularity|String|The granularity of the data inside the segment. E.g. a value of "minute" will mean that data is aggregated at minutely granularity. That is, if there are collisions in the tuple (minute(timestamp), dimensions), then it will aggregate values together using the aggregators instead of storing individual rows.|yes| |indexGranularity|String|The granularity of the data inside the segment. E.g. a value of "minute" will mean that data is aggregated at minutely granularity. That is, if there are collisions in the tuple (minute(timestamp), dimensions), then it will aggregate values together using the aggregators instead of storing individual rows.|yes|
|shardSpec|Object|This describes the shard that is represented by this server. This must be specified properly in order to have multiple realtime nodes indexing the same data stream in a sharded fashion.|no| |shardSpec|Object|This describes the shard that is represented by this server. This must be specified properly in order to have multiple realtime nodes indexing the same data stream in a sharded fashion.|no|
### Config ### Config
This provides configuration for the data processing portion of the realtime stream processor. This provides configuration for the data processing portion of the realtime stream processor.
@ -101,6 +106,22 @@ 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| |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| |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
Firehoses describe the data stream source. See [Firehose](Firehose.html) for more information on firehose configuration.
### Plumber
The Plumber handles generated segments both while they are being generated and when they are "done". The configuration parameters in the example are:
* `type` specifies the type of plumber in terms of configuration schema. The Plumber configuration in the example is for the often-used RealtimePlumber.
* `windowPeriod` is the amount of lag time to allow events. The example configures a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server.
* `segmentGranularity` specifies the granularity of the segment, or the amount of time a segment will represent.
* `basePersistDirectory` is the directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists.
* `rejectionPolicy` determines what events are rejected upon ingestion.
See [Plumber](Plumber.html) for a fuller discussion of Plumber configuration.
Constraints Constraints
----------- -----------

View File

@ -37,7 +37,7 @@ There are several main parts to a search query:
|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
|searchDimensions|The dimensions to run the search over. Excluding this means the search is run over all dimensions.|no| |searchDimensions|The dimensions to run the search over. Excluding this means the search is run over all dimensions.|no|
|query|See [SearchQuerySpec](SearchQuerySpec.html).|yes| |query|See [SearchQuerySpec](SearchQuerySpec.html).|yes|
|sort|How the results of the search should sorted. Two possible types here are "lexicographic" and "strlen".|yes| |sort|How the results of the search should be sorted. Two possible types here are "lexicographic" and "strlen".|yes|
|context|An additional JSON Object which can be used to specify certain flags.|no| |context|An additional JSON Object which can be used to specify certain flags.|no|
The format of the result is: The format of the result is:

View File

@ -15,7 +15,7 @@ Segment metadata queries return per segment information about:
{ {
"queryType":"segmentMetadata", "queryType":"segmentMetadata",
"dataSource":"sample_datasource", "dataSource":"sample_datasource",
"intervals":["2013-01-01/2014-01-01"], "intervals":["2013-01-01/2014-01-01"]
} }
``` ```

142
docs/content/SelectQuery.md Normal file
View File

@ -0,0 +1,142 @@
---
layout: doc_page
---
# Select Queries
Select queries return raw Druid rows and support pagination.
```json
{
"queryType": "select",
"dataSource": "wikipedia",
"dimensions":[],
"metrics":[],
"granularity": "all",
"intervals": [
"2013-01-01/2013-01-02"
],
"pagingSpec":{"pagingIdentifiers": {}, "threshold":5}
}
```
There are several main parts to a select query:
|property|description|required?|
|--------|-----------|---------|
|queryType|This String should always be "select"; this is the first thing Druid looks at to figure out how to interpret the query|yes|
|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes|
|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
|dimensions|The list of dimensions to select. If left empty, all dimensions are returned.|no|
|metrics|The list of metrics to select. If left empty, all metrics are returned.|no|
|pagingSpec|A JSON object indicating offsets into different scanned segments. Select query results will return a pagingSpec that can be reused for pagination.|yes|
|context|An additional JSON Object which can be used to specify certain flags.|no|
The format of the result is:
```json
[{
"timestamp" : "2013-01-01T00:00:00.000Z",
"result" : {
"pagingIdentifiers" : {
"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" : 4
},
"events" : [ {
"segmentId" : "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 0,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "1",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "11._korpus_(NOVJ)",
"language" : "sl",
"newpage" : "0",
"user" : "EmausBot",
"count" : 1.0,
"added" : 39.0,
"delta" : 39.0,
"variation" : 39.0,
"deleted" : 0.0
}
}, {
"segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 1,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "0",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "112_U.S._580",
"language" : "en",
"newpage" : "1",
"user" : "MZMcBride",
"count" : 1.0,
"added" : 70.0,
"delta" : 70.0,
"variation" : 70.0,
"deleted" : 0.0
}
}, {
"segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 2,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "0",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "113_U.S._243",
"language" : "en",
"newpage" : "1",
"user" : "MZMcBride",
"count" : 1.0,
"added" : 77.0,
"delta" : 77.0,
"variation" : 77.0,
"deleted" : 0.0
}
}, {
"segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 3,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "0",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "113_U.S._73",
"language" : "en",
"newpage" : "1",
"user" : "MZMcBride",
"count" : 1.0,
"added" : 70.0,
"delta" : 70.0,
"variation" : 70.0,
"deleted" : 0.0
}
}, {
"segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 4,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "0",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "113_U.S._756",
"language" : "en",
"newpage" : "1",
"user" : "MZMcBride",
"count" : 1.0,
"added" : 68.0,
"delta" : 68.0,
"variation" : 68.0,
"deleted" : 0.0
}
} ]
}
} ]
```
The result returns a global pagingSpec that can be reused for the next select query. The offset will need to be increased by 1 on the client side.

View File

@ -2,14 +2,14 @@
layout: doc_page layout: doc_page
--- ---
# Tasks # Tasks
Tasks are run on middle managers and always operate on a single data source. Tasks are run on middle managers and always operate on a single data source. Tasks are submitted using [POST requests](Indexing-Service.html).
There are several different types of tasks. There are several different types of tasks.
Segment Creation 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: 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,16 +50,16 @@ The Index Task is a simpler variation of the Index Hadoop task that is designed
|property|description|required?| |property|description|required?|
|--------|-----------|---------| |--------|-----------|---------|
|type|The task type, this should always be "index".|yes| |type|The task type, this should always be "index".|yes|
|id|The task ID.|no| |id|The task ID. If this is not explicitly specified, Druid generates the task ID using the name of the task file and date-time stamp. |no|
|granularitySpec|See [granularitySpec](Tasks.html)|yes| |granularitySpec|Specifies the segment chunks that the task will process. `type` is always "uniform"; `gran` sets the granularity of the chunks ("DAY" means all segments containing timestamps in the same day), while `intervals` sets the interval that the chunks will cover.|yes|
|spatialDimensions|Dimensions to build spatial indexes over. See [Spatial-Indexing](Spatial-Indexing.html)|no| |spatialDimensions|Dimensions to build spatial indexes over. See [Geographic Queries](GeographicQueries.html).|no|
|aggregators|The metrics to aggregate in the data set. For more info, see [Aggregations](Aggregations.html)|yes| |aggregators|The metrics to aggregate in the data set. For more info, see [Aggregations](Aggregations.html).|yes|
|indexGranularity|The rollup granularity for timestamps.|no| |indexGranularity|The rollup granularity for timestamps. See [Realtime Ingestion](Realtime-ingestion.html) for more information. |no|
|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|no| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|no|
|firehose|The input source of data. For more info, see [Firehose](Firehose.html)|yes| |firehose|The input source of data. For more info, see [Firehose](Firehose.html).|yes|
|rowFlushBoundary|Used in determining when intermediate persist should occur to disk.|no| |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. The Hadoop Index Task is used to index larger data sets that require the parallelization and processing power of a Hadoop cluster.
@ -74,16 +74,16 @@ The Hadoop Index Task is used to index larger data sets that require the paralle
|--------|-----------|---------| |--------|-----------|---------|
|type|The task type, this should always be "index_hadoop".|yes| |type|The task type, this should always be "index_hadoop".|yes|
|config|A Hadoop Index Config. See [Batch Ingestion](Batch-ingestion.html)|yes| |config|A Hadoop Index Config. See [Batch Ingestion](Batch-ingestion.html)|yes|
|hadoopCoordinates|The Maven \<groupId\>:\<artifactId\>:\<version\> of Hadoop to use. The default is "org.apache.hadoop:hadoop-core:1.0.3".|no| |hadoopCoordinates|The Maven \<groupId\>:\<artifactId\>:\<version\> of Hadoop to use. The default is "org.apache.hadoop:hadoop-client:2.3.0".|no|
The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopBatchIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `metadataUpdateSpec`. The Indexing Service takes care of setting these fields internally. 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`, `metadataUpdateSpec`. 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. Druid is compiled against Apache hadoop-client 2.3.0. However, if you happen to use a different flavor of hadoop that is API compatible with hadoop-client 2.3.0, you should only have to change the hadoopCoordinates property to point to the maven artifact used by your distribution.
##### Resolving dependency conflicts running HadoopIndexTask #### 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. 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.
@ -91,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. 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: 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:
@ -163,30 +163,32 @@ The indexing service can also run real-time tasks. These tasks effectively trans
|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| |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| |requiredCapacity|Integer|How much middle manager capacity this task will take.|yes|
For schema, fireDepartmentConfig, windowPeriod, segmentGranularity, and rejectionPolicy, see the [realtime-ingestion doc](Realtime-ingestion.html). For firehose configuration, see [Firehose](Firehose.html). For schema, windowPeriod, segmentGranularity, and other configuration information, see [Realtime Ingestion](Realtime-ingestion.html). For firehose configuration, see [Firehose](Firehose.html).
Segment Merging Tasks 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: Append tasks append a list of segments together into a single segment (one after the other). The grammar is:
```json ```json
{ {
"type": "append",
"id": <task_id>, "id": <task_id>,
"dataSource": <task_datasource>, "dataSource": <task_datasource>,
"segments": <JSON list of DataSegment objects to append> "segments": <JSON list of DataSegment objects to append>
} }
``` ```
#### Merge Task ### Merge Task
Merge tasks merge a list of segments together. Any common timestamps are merged. The grammar is: Merge tasks merge a list of segments together. Any common timestamps are merged. The grammar is:
```json ```json
{ {
"type": "merge",
"id": <task_id>, "id": <task_id>,
"dataSource": <task_datasource>, "dataSource": <task_datasource>,
"segments": <JSON list of DataSegment objects to append> "segments": <JSON list of DataSegment objects to append>
@ -196,24 +198,26 @@ Merge tasks merge a list of segments together. Any common timestamps are merged.
Segment Destroying Tasks Segment Destroying Tasks
------------------------ ------------------------
#### Delete Task ### Delete Task
Delete tasks create empty segments with no data. The grammar is: Delete tasks create empty segments with no data. The grammar is:
```json ```json
{ {
"type": "delete",
"id": <task_id>, "id": <task_id>,
"dataSource": <task_datasource>, "dataSource": <task_datasource>,
"segments": <JSON list of DataSegment objects to append> "segments": <JSON list of DataSegment objects to append>
} }
``` ```
#### 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: 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:
```json ```json
{ {
"type": "kill",
"id": <task_id>, "id": <task_id>,
"dataSource": <task_datasource>, "dataSource": <task_datasource>,
"segments": <JSON list of DataSegment objects to append> "segments": <JSON list of DataSegment objects to append>
@ -223,12 +227,13 @@ Kill tasks delete all information about a segment and removes it from deep stora
Misc. Tasks 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: These tasks convert segments from an existing older index version to the latest index version. The available grammar is:
```json ```json
{ {
"type": "version_converter",
"id": <task_id>, "id": <task_id>,
"groupId" : <task_group_id>, "groupId" : <task_group_id>,
"dataSource": <task_datasource>, "dataSource": <task_datasource>,
@ -237,12 +242,13 @@ 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: These tasks start, sleep for a time and are used only for testing. The available grammar is:
```json ```json
{ {
"type": "noop",
"id": <optional_task_id>, "id": <optional_task_id>,
"interval" : <optional_segment_interval>, "interval" : <optional_segment_interval>,
"runTime" : <optional_millis_to_sleep>, "runTime" : <optional_millis_to_sleep>,

View File

@ -9,6 +9,7 @@ TopN queries return a sorted set of results for the values in a given dimension
A topN query object looks like: A topN query object looks like:
```json ```json
{
"queryType": "topN", "queryType": "topN",
"dataSource": "sample_data", "dataSource": "sample_data",
"dimension": "sample_dim", "dimension": "sample_dim",

View File

@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu
### Download a Tarball ### Download a Tarball
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.69-bin.tar.gz). Download this file to a directory of your choosing. We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz). Download this file to a directory of your choosing.
You can extract the awesomeness within by issuing: 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: Not too lost so far right? That's great! If you cd into the directory:
``` ```
cd druid-services-0.6.69 cd druid-services-0.6.101
``` ```
You should see a bunch of files: You should see a bunch of files:

View File

@ -42,7 +42,7 @@ Metrics (things to aggregate over):
Setting Up Setting Up
---------- ----------
At this point, you should already have Druid downloaded and are comfortable with running a Druid cluster locally. If you are not, see [here](Tutiroal%3A-The-Druid-Cluster.html). At this point, you should already have Druid downloaded and are comfortable with running a Druid cluster locally. If you are not, see [here](Tutorial%3A-The-Druid-Cluster.html).
Let's start from our usual starting point in the tarball directory. Let's start from our usual starting point in the tarball directory.
@ -136,7 +136,7 @@ Indexing the Data
To index the data and build a Druid segment, we are going to need to submit a task to the indexing service. This task should already exist: To index the data and build a Druid segment, we are going to need to submit a task to the indexing service. This task should already exist:
``` ```
examples/indexing/index_task.json examples/indexing/wikipedia_index_task.json
``` ```
Open up the file to see the following: Open up the file to see the following:

View File

@ -167,7 +167,7 @@ You should be comfortable starting Druid nodes at this point. If not, it may be
] ]
``` ```
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. Note: This config uses a "test" [rejection policy](Plumber.html) 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 3. Let's copy and paste some data into the Kafka console producer

View File

@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende
If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. 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.69-bin.tar.gz) You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz)
and untar the contents within by issuing: and untar the contents within by issuing:
@ -149,7 +149,7 @@ druid.port=8081
druid.zk.service.host=localhost druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.69"] druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101"]
# Dummy read only AWS account (used to download example data) # Dummy read only AWS account (used to download example data)
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
@ -240,7 +240,7 @@ druid.port=8083
druid.zk.service.host=localhost druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.69","io.druid.extensions:druid-kafka-seven:0.6.69"] druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.101","io.druid.extensions:druid-kafka-seven:0.6.101"]
# Change this config to db to hand off to the rest of the Druid cluster # Change this config to db to hand off to the rest of the Druid cluster
druid.publish.type=noop druid.publish.type=noop

View File

@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu
h3. Download a Tarball 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.69-bin.tar.gz) We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz)
Download this file to a directory of your choosing. Download this file to a directory of your choosing.
You can extract the awesomeness within by issuing: 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: Not too lost so far right? That's great! If you cd into the directory:
``` ```
cd druid-services-0.6.69 cd druid-services-0.6.101
``` ```
You should see a bunch of files: You should see a bunch of files:

View File

@ -1,77 +1,93 @@
--- ---
layout: doc_page layout: doc_page
--- ---
Greetings! We see you've taken an interest in Druid. That's awesome! Hopefully this tutorial will help clarify some core Druid concepts. We will go through one of the Real-time "Examples":Examples.html, and issue some basic Druid queries. The data source we'll be working with is the "Twitter spritzer stream":https://dev.twitter.com/docs/streaming-apis/streams/public. If you are ready to explore Druid, brave its challenges, and maybe learn a thing or two, read on! Greetings! We see you've taken an interest in Druid. That's awesome! Hopefully this tutorial will help clarify some core Druid concepts. We will go through one of the Real-time [Examples](Examples.html), and issue some basic Druid queries. The data source we'll be working with is the [Twitter spritzer stream](https://dev.twitter.com/docs/streaming-apis/streams/public). If you are ready to explore Druid, brave its challenges, and maybe learn a thing or two, read on!
h2. Setting Up # Setting Up
There are two ways to setup Druid: download a tarball, or build it from source. There are two ways to setup Druid: download a tarball, or build it from source.
h3. Download a Tarball # Download a Tarball
We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.69-bin.tar.gz. We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.101-bin.tar.gz).
Download this bad boy to a directory of your choosing. Download this bad boy to a directory of your choosing.
You can extract the awesomeness within by issuing: You can extract the awesomeness within by issuing:
pre. tar -zxvf druid-services-0.X.X.tar.gz ```
tar -zxvf druid-services-0.X.X.tar.gz
```
Not too lost so far right? That's great! If you cd into the directory: Not too lost so far right? That's great! If you cd into the directory:
pre. cd druid-services-0.X.X ```
cd druid-services-0.X.X
```
You should see a bunch of files: You should see a bunch of files:
* run_example_server.sh * run_example_server.sh
* run_example_client.sh * run_example_client.sh
* LICENSE, config, examples, lib directories * LICENSE, config, examples, lib directories
h3. Clone and Build from Source # Clone and Build from Source
The other way to setup Druid is from source via git. To do so, run these commands: The other way to setup Druid is from source via git. To do so, run these commands:
<pre><code>git clone git@github.com:metamx/druid.git ```
git clone git@github.com:metamx/druid.git
cd druid cd druid
git checkout druid-0.X.X git checkout druid-0.X.X
./build.sh ./build.sh
</code></pre> ```
You should see a bunch of files: You should see a bunch of files:
<pre><code>DruidCorporateCLA.pdf README common examples indexer pom.xml server ```
DruidCorporateCLA.pdf README common examples indexer pom.xml server
DruidIndividualCLA.pdf build.sh doc group_by.body install publications services DruidIndividualCLA.pdf build.sh doc group_by.body install publications services
LICENSE client eclipse_formatting.xml index-common merger realtime LICENSE client eclipse_formatting.xml index-common merger realtime
</code></pre> ```
You can find the example executables in the examples/bin directory: You can find the example executables in the examples/bin directory:
* run_example_server.sh * run_example_server.sh
* run_example_client.sh * run_example_client.sh
h2. Running Example Scripts # Running Example Scripts
Let's start doing stuff. You can start a Druid "Realtime":Realtime.html node by issuing: Let's start doing stuff. You can start a Druid [Realtime](Realtime.html) node by issuing:
<code>./run_example_server.sh</code>
```
./run_example_server.sh
```
Select "twitter". Select "twitter".
You'll need to register a new application with the twitter API, which only takes a minute. Go to "https://twitter.com/oauth_clients/new":https://twitter.com/oauth_clients/new and fill out the form and submit. Don't worry, the home page and callback url can be anything. This will generate keys for the Twitter example application. Take note of the values for consumer key/secret and access token/secret. You'll need to register a new application with the twitter API, which only takes a minute. Go to [this link](https://twitter.com/oauth_clients/new":https://twitter.com/oauth_clients/new) and fill out the form and submit. Don't worry, the home page and callback url can be anything. This will generate keys for the Twitter example application. Take note of the values for consumer key/secret and access token/secret.
Enter your credentials when prompted. Enter your credentials when prompted.
Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below. If you see crazy exceptions, you probably typed in your login information incorrectly. Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below. If you see crazy exceptions, you probably typed in your login information incorrectly.
<pre><code>2013-05-17 23:04:40,934 INFO [main] org.mortbay.log - Started SelectChannelConnector@0.0.0.0:8080
```
2013-05-17 23:04:40,934 INFO [main] org.mortbay.log - Started SelectChannelConnector@0.0.0.0:8080
2013-05-17 23:04:40,935 INFO [main] com.metamx.common.lifecycle.Lifecycle$AnnotationBasedHandler - Invoking start method[public void com.metamx.druid.http.FileRequestLogger.start()] on object[com.metamx.druid.http.FileRequestLogger@42bb0406]. 2013-05-17 23:04:40,935 INFO [main] com.metamx.common.lifecycle.Lifecycle$AnnotationBasedHandler - Invoking start method[public void com.metamx.druid.http.FileRequestLogger.start()] on object[com.metamx.druid.http.FileRequestLogger@42bb0406].
2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] twitter4j.TwitterStreamImpl - Connection established. 2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] twitter4j.TwitterStreamImpl - Connection established.
2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] io.druid.examples.twitter.TwitterSpritzerFirehoseFactory - Connected_to_Twitter 2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] io.druid.examples.twitter.TwitterSpritzerFirehoseFactory - Connected_to_Twitter
2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] twitter4j.TwitterStreamImpl - Receiving status stream. 2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] twitter4j.TwitterStreamImpl - Receiving status stream.
</code></pre> ```
Periodically, you'll also see messages of the form: Periodically, you'll also see messages of the form:
<pre><code>2013-05-17 23:04:59,793 INFO [chief-twitterstream] io.druid.examples.twitter.TwitterSpritzerFirehoseFactory - nextRow() has returned 1,000 InputRows
</code></pre> ```
2013-05-17 23:04:59,793 INFO [chief-twitterstream] io.druid.examples.twitter.TwitterSpritzerFirehoseFactory - nextRow() has returned 1,000 InputRows
```
These messages indicate you are ingesting events. The Druid real time-node ingests events in an in-memory buffer. Periodically, these events will be persisted to disk. Persisting to disk generates a whole bunch of logs: These messages indicate you are ingesting events. The Druid real time-node ingests events in an in-memory buffer. Periodically, these events will be persisted to disk. Persisting to disk generates a whole bunch of logs:
<pre><code>2013-05-17 23:06:40,918 INFO [chief-twitterstream] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - Submitting persist runnable for dataSource[twitterstream] ```
2013-05-17 23:06:40,918 INFO [chief-twitterstream] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - Submitting persist runnable for dataSource[twitterstream]
2013-05-17 23:06:40,920 INFO [twitterstream-incremental-persist] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - DataSource[twitterstream], Interval[2013-05-17T23:00:00.000Z/2013-05-18T00:00:00.000Z], persisting Hydrant[FireHydrant{index=com.metamx.druid.index.v1.IncrementalIndex@126212dd, queryable=com.metamx.druid.index.IncrementalIndexSegment@64c47498, count=0}] 2013-05-17 23:06:40,920 INFO [twitterstream-incremental-persist] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - DataSource[twitterstream], Interval[2013-05-17T23:00:00.000Z/2013-05-18T00:00:00.000Z], persisting Hydrant[FireHydrant{index=com.metamx.druid.index.v1.IncrementalIndex@126212dd, queryable=com.metamx.druid.index.IncrementalIndexSegment@64c47498, count=0}]
2013-05-17 23:06:40,937 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Starting persist for interval[2013-05-17T23:00:00.000Z/2013-05-17T23:07:00.000Z], rows[4,666] 2013-05-17 23:06:40,937 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Starting persist for interval[2013-05-17T23:00:00.000Z/2013-05-17T23:07:00.000Z], rows[4,666]
2013-05-17 23:06:41,039 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - outDir[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] completed index.drd in 11 millis. 2013-05-17 23:06:41,039 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - outDir[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] completed index.drd in 11 millis.
@ -88,16 +104,20 @@ These messages indicate you are ingesting events. The Druid real time-node inges
2013-05-17 23:06:41,425 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexIO$DefaultIndexIOHandler - Converting v8[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] to v9[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0] 2013-05-17 23:06:41,425 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexIO$DefaultIndexIOHandler - Converting v8[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] to v9[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0]
2013-05-17 23:06:41,426 INFO [twitterstream-incremental-persist] 2013-05-17 23:06:41,426 INFO [twitterstream-incremental-persist]
... ETC ... ETC
</code></pre> ```
The logs are about building different columns, probably not the most exciting stuff (they might as well be in Vulcan) if are you learning about Druid for the first time. Nevertheless, if you are interested in the details of our real-time architecture and why we persist indexes to disk, I suggest you read our "White Paper":http://static.druid.io/docs/druid.pdf. The logs are about building different columns, probably not the most exciting stuff (they might as well be in Vulcan) if are you learning about Druid for the first time. Nevertheless, if you are interested in the details of our real-time architecture and why we persist indexes to disk, I suggest you read our "White Paper":http://static.druid.io/docs/druid.pdf.
Okay, things are about to get real (-time). To query the real-time node you've spun up, you can issue: Okay, things are about to get real (-time). To query the real-time node you've spun up, you can issue:
<pre>./run_example_client.sh</pre>
Select "twitter" once again. This script issues ["GroupByQuery":GroupByQuery.html]s to the twitter data we've been ingesting. The query looks like this: ```
./run_example_client.sh
```
<pre><code>{ Select "twitter" once again. This script issues [GroupByQueries](GroupByQuery.html) to the twitter data we've been ingesting. The query looks like this:
```json
{
"queryType": "groupBy", "queryType": "groupBy",
"dataSource": "twitterstream", "dataSource": "twitterstream",
"granularity": "all", "granularity": "all",
@ -109,13 +129,14 @@ Select "twitter" once again. This script issues ["GroupByQuery":GroupByQuery.htm
"filter": { "type": "selector", "dimension": "lang", "value": "en" }, "filter": { "type": "selector", "dimension": "lang", "value": "en" },
"intervals":["2012-10-01T00:00/2020-01-01T00"] "intervals":["2012-10-01T00:00/2020-01-01T00"]
} }
</code></pre> ```
This is a **groupBy** query, which you may be familiar with from SQL. We are grouping, or aggregating, via the **dimensions** field: ["lang", "utc_offset"]. We are **filtering** via the **"lang"** dimension, to only look at english tweets. Our **aggregations** are what we are calculating: a row count, and the sum of the tweets in our data. This is a **groupBy** query, which you may be familiar with from SQL. We are grouping, or aggregating, via the **dimensions** field: ["lang", "utc_offset"]. We are **filtering** via the **"lang"** dimension, to only look at english tweets. Our **aggregations** are what we are calculating: a row count, and the sum of the tweets in our data.
The result looks something like this: The result looks something like this:
<pre><code>[ ```json
[
{ {
"version": "v1", "version": "v1",
"timestamp": "2012-10-01T00:00:00.000Z", "timestamp": "2012-10-01T00:00:00.000Z",
@ -137,41 +158,48 @@ The result looks something like this:
} }
}, },
... ...
</code></pre> ```
This data, plotted in a time series/distribution, looks something like this: This data, plotted in a time series/distribution, looks something like this:
!http://metamarkets.com/wp-content/uploads/2013/06/tweets_timezone_offset.png(Timezone / Tweets Scatter Plot)! ![Tweets](http://metamarkets.com/wp-content/uploads/2013/06/tweets_timezone_offset.png)
This groupBy query is a bit complicated and we'll return to it later. For the time being, just make sure you are getting some blocks of data back. If you are having problems, make sure you have "curl":http://curl.haxx.se/ installed. Control+C to break out of the client script. This groupBy query is a bit complicated and we'll return to it later. For the time being, just make sure you are getting some blocks of data back. If you are having problems, make sure you have [curl](http://curl.haxx.se/) installed. Control+C to break out of the client script.
h2. Querying Druid # Querying Druid
In your favorite editor, create the file: In your favorite editor, create the file:
<pre>time_boundary_query.body</pre>
```
time_boundary_query.body
```
Druid queries are JSON blobs which are relatively painless to create programmatically, but an absolute pain to write by hand. So anyway, we are going to create a Druid query by hand. Add the following to the file you just created: Druid queries are JSON blobs which are relatively painless to create programmatically, but an absolute pain to write by hand. So anyway, we are going to create a Druid query by hand. Add the following to the file you just created:
<pre><code>{
```json
{
"queryType" : "timeBoundary", "queryType" : "timeBoundary",
"dataSource" : "twitterstream" "dataSource" : "twitterstream"
} }
</code></pre> ```
The "TimeBoundaryQuery":TimeBoundaryQuery.html is one of the simplest Druid queries. To run the query, you can issue: The "TimeBoundaryQuery":TimeBoundaryQuery.html is one of the simplest Druid queries. To run the query, you can issue:
<pre><code>
```
curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json' -d @time_boundary_query.body curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json' -d @time_boundary_query.body
</code></pre> ```
We get something like this JSON back: We get something like this JSON back:
<pre><code>[ { ```json
{
"timestamp" : "2013-06-10T19:09:00.000Z", "timestamp" : "2013-06-10T19:09:00.000Z",
"result" : { "result" : {
"minTime" : "2013-06-10T19:09:00.000Z", "minTime" : "2013-06-10T19:09:00.000Z",
"maxTime" : "2013-06-10T20:50:00.000Z" "maxTime" : "2013-06-10T20:50:00.000Z"
} }
} ] } ]
</code></pre> ```
That's the result. What information do you think the result is conveying? That's the result. What information do you think the result is conveying?
... ...
@ -179,11 +207,14 @@ If you said the result is indicating the maximum and minimum timestamps we've se
Return to your favorite editor and create the file: Return to your favorite editor and create the file:
<pre>timeseries_query.body</pre> ```
timeseries_query.body
```
We are going to make a slightly more complicated query, the "TimeseriesQuery":TimeseriesQuery.html. Copy and paste the following into the file: We are going to make a slightly more complicated query, the [TimeseriesQuery](TimeseriesQuery.html). Copy and paste the following into the file:
<pre><code>{ ```json
{
"queryType":"timeseries", "queryType":"timeseries",
"dataSource":"twitterstream", "dataSource":"twitterstream",
"intervals":["2010-01-01/2020-01-01"], "intervals":["2010-01-01/2020-01-01"],
@ -193,22 +224,26 @@ We are going to make a slightly more complicated query, the "TimeseriesQuery":Ti
{ "type": "doubleSum", "fieldName": "tweets", "name": "tweets"} { "type": "doubleSum", "fieldName": "tweets", "name": "tweets"}
] ]
} }
</code></pre> ```
You are probably wondering, what are these "Granularities":Granularities.html and "Aggregations":Aggregations.html things? What the query is doing is aggregating some metrics over some span of time. You are probably wondering, what are these [Granularities](Granularities.html) and [Aggregations](Aggregations.html) things? What the query is doing is aggregating some metrics over some span of time.
To issue the query and get some results, run the following in your command line: To issue the query and get some results, run the following in your command line:
<pre><code>curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json' -d @timeseries_query.body</code></pre>
```
curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json' -d @timeseries_query.body
```
Once again, you should get a JSON blob of text back with your results, that looks something like this: Once again, you should get a JSON blob of text back with your results, that looks something like this:
<pre><code>[ { ```json
[ {
"timestamp" : "2013-06-10T19:09:00.000Z", "timestamp" : "2013-06-10T19:09:00.000Z",
"result" : { "result" : {
"tweets" : 358562.0, "tweets" : 358562.0,
"rows" : 272271 "rows" : 272271
} }
} ] } ]
</code></pre> ```
If you issue the query again, you should notice your results updating. If you issue the query again, you should notice your results updating.
@ -216,7 +251,8 @@ Right now all the results you are getting back are being aggregated into a singl
If you loudly exclaimed "we can change granularity to minute", you are absolutely correct again! We can specify different granularities to bucket our results, like so: If you loudly exclaimed "we can change granularity to minute", you are absolutely correct again! We can specify different granularities to bucket our results, like so:
<pre><code>{ ```json
{
"queryType":"timeseries", "queryType":"timeseries",
"dataSource":"twitterstream", "dataSource":"twitterstream",
"intervals":["2010-01-01/2020-01-01"], "intervals":["2010-01-01/2020-01-01"],
@ -226,11 +262,12 @@ If you loudly exclaimed "we can change granularity to minute", you are absolutel
{ "type": "doubleSum", "fieldName": "tweets", "name": "tweets"} { "type": "doubleSum", "fieldName": "tweets", "name": "tweets"}
] ]
} }
</code></pre> ```
This gives us something like the following: This gives us something like the following:
<pre><code>[ { ```json
[ {
"timestamp" : "2013-06-10T19:09:00.000Z", "timestamp" : "2013-06-10T19:09:00.000Z",
"result" : { "result" : {
"tweets" : 2650.0, "tweets" : 2650.0,
@ -250,16 +287,21 @@ This gives us something like the following:
} }
}, },
... ...
</code></pre> ```
h2. Solving a Problem # Solving a Problem
One of Druid's main powers (see what we did there?) is to provide answers to problems, so let's pose a problem. What if we wanted to know what the top hash tags are, ordered by the number tweets, where the language is english, over the last few minutes you've been reading this tutorial? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the "GroupByQuery":GroupByQuery.html. It would be nice if we could group by results by dimension value and somehow sort those results... and it turns out we can! One of Druid's main powers (see what we did there?) is to provide answers to problems, so let's pose a problem. What if we wanted to know what the top hash tags are, ordered by the number tweets, where the language is english, over the last few minutes you've been reading this tutorial? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the "GroupByQuery":GroupByQuery.html. It would be nice if we could group by results by dimension value and somehow sort those results... and it turns out we can!
Let's create the file: Let's create the file:
<pre>group_by_query.body</pre>
```
group_by_query.body
```
and put the following in there: and put the following in there:
<pre><code>{
```json
{
"queryType": "groupBy", "queryType": "groupBy",
"dataSource": "twitterstream", "dataSource": "twitterstream",
"granularity": "all", "granularity": "all",
@ -271,16 +313,20 @@ and put the following in there:
"filter": {"type": "selector", "dimension": "lang", "value": "en" }, "filter": {"type": "selector", "dimension": "lang", "value": "en" },
"intervals":["2012-10-01T00:00/2020-01-01T00"] "intervals":["2012-10-01T00:00/2020-01-01T00"]
} }
</code></pre> ```
Woah! Our query just got a way more complicated. Now we have these "Filters":Filters.html things and this "OrderBy":OrderBy.html thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. Woah! Our query just got a way more complicated. Now we have these "Filters":Filters.html things and this "OrderBy":OrderBy.html thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question.
If you issue the query: If you issue the query:
<pre><code>curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json' -d @group_by_query.body</code></pre>
```
curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json' -d @group_by_query.body
```
You should hopefully see an answer to our question. For my twitter stream, it looks like this: You should hopefully see an answer to our question. For my twitter stream, it looks like this:
<pre><code>[ { ```json
[ {
"version" : "v1", "version" : "v1",
"timestamp" : "2012-10-01T00:00:00.000Z", "timestamp" : "2012-10-01T00:00:00.000Z",
"event" : { "event" : {
@ -316,12 +362,12 @@ You should hopefully see an answer to our question. For my twitter stream, it lo
"htags" : "IDidntTextYouBackBecause" "htags" : "IDidntTextYouBackBecause"
} }
} ] } ]
</code></pre> ```
Feel free to tweak other query parameters to answer other questions you may have about the data. Feel free to tweak other query parameters to answer other questions you may have about the data.
h2. Additional Information # Additional Information
This tutorial is merely showcasing a small fraction of what Druid can do. Next, continue on to "The Druid Cluster":./Tutorial:-The-Druid-Cluster.html. This tutorial is merely showcasing a small fraction of what Druid can do. Next, continue on to [The Druid Cluster](./Tutorial:-The-Druid-Cluster.html).
And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our "google groups page":http://www.groups.google.com/forum/#!forum/druid-development. And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](http://www.groups.google.com/forum/#!forum/druid-development).

View File

@ -31,20 +31,20 @@ We have more details about the general design of the system and why you might wa
When Druid? When Druid?
---------- ----------
* You need to do interactive, fast, exploration of large amounts of data * You need to do interactive, fast, exploration on large amounts of data
* You need analytics (not key value store) * You need analytics (not a key-value store)
* You have a lot of data (10s of Billions of events added per day, 10s of TB of data added per day) * You have a lot of data (10s of billions of events added per day, 10s of TB of data added per day)
* You want to do your analysis on data as its happening (realtime) * You want to do your analysis on data as its happening (in real-time)
* Your store needs to be always-on, 24x7x365 and years into the future. * You need a data store that is always available, 24x7x365, and years into the future.
Not Druid? Not Druid?
---------- ----------
* The amount of data you have can easily be handled by MySql * The amount of data you have can easily be handled by MySQL
* Your querying for individual entries or doing lookups (Not Analytics) * You're querying for individual entries or doing lookups (not analytics)
* Batch is good enough * Batch ingestion is good enough
* Canned queries is good enough * Canned queries are good enough
* Downtime is no big deal * Downtime is no big deal

View File

@ -22,12 +22,6 @@ h2. Configuration
* "Broker":Broker-Config.html * "Broker":Broker-Config.html
* "Indexing Service":Indexing-Service-Config.html * "Indexing Service":Indexing-Service-Config.html
h2. Operations
* "Extending Druid":./Modules.html
* "Cluster Setup":./Cluster-setup.html
* "Booting a Production Cluster":./Booting-a-production-cluster.html
* "Performance FAQ":./Performance-FAQ.html
h2. Data Ingestion h2. Data Ingestion
* "Realtime":./Realtime-ingestion.html * "Realtime":./Realtime-ingestion.html
* "Batch":./Batch-ingestion.html * "Batch":./Batch-ingestion.html
@ -36,6 +30,12 @@ h2. Data Ingestion
* "Data Formats":./Data_formats.html * "Data Formats":./Data_formats.html
* "Ingestion FAQ":./Ingestion-FAQ.html * "Ingestion FAQ":./Ingestion-FAQ.html
h2. Operations
* "Extending Druid":./Modules.html
* "Cluster Setup":./Cluster-setup.html
* "Booting a Production Cluster":./Booting-a-production-cluster.html
* "Performance FAQ":./Performance-FAQ.html
h2. Querying h2. Querying
* "Querying":./Querying.html * "Querying":./Querying.html
** "Filters":./Filters.html ** "Filters":./Filters.html
@ -75,6 +75,7 @@ h2. Architecture
h2. Experimental h2. Experimental
* "About Experimental Features":./About-Experimental-Features.html * "About Experimental Features":./About-Experimental-Features.html
* "Geographic Queries":./GeographicQueries.html * "Geographic Queries":./GeographicQueries.html
* "Select Query":./SelectQuery.html
h2. Development h2. Development
* "Versioning":./Versioning.html * "Versioning":./Versioning.html

View File

@ -4,7 +4,7 @@ druid.port=8081
druid.zk.service.host=localhost druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.69"] druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.101"]
# Dummy read only AWS account (used to download example data) # Dummy read only AWS account (used to download example data)
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b

View File

@ -4,7 +4,7 @@ druid.port=8083
druid.zk.service.host=localhost druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.69","io.druid.extensions:druid-kafka-seven:0.6.69","io.druid.extensions:druid-rabbitmq:0.6.69"] druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.101","io.druid.extensions:druid-kafka-seven:0.6.101","io.druid.extensions:druid-rabbitmq:0.6.101"]
# Change this config to db to hand off to the rest of the Druid cluster # Change this config to db to hand off to the rest of the Druid cluster
druid.publish.type=noop druid.publish.type=noop

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.70-SNAPSHOT</version> <version>0.6.102-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>
@ -58,6 +58,11 @@
<artifactId>twitter4j-stream</artifactId> <artifactId>twitter4j-stream</artifactId>
<version>3.0.3</version> <version>3.0.3</version>
</dependency> </dependency>
<dependency>
<groupId>commons-validator</groupId>
<artifactId>commons-validator</artifactId>
<version>1.4.0</version>
</dependency>
<!-- For tests! --> <!-- For tests! -->
<dependency> <dependency>
@ -82,14 +87,14 @@
${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar
</outputFile> </outputFile>
<filters> <filters>
<filter> <filter>
<artifact>*:*</artifact> <artifact>*:*</artifact>
<excludes> <excludes>
<exclude>META-INF/*.SF</exclude> <exclude>META-INF/*.SF</exclude>
<exclude>META-INF/*.DSA</exclude> <exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude> <exclude>META-INF/*.RSA</exclude>
</excludes> </excludes>
</filter> </filter>
</filters> </filters>
</configuration> </configuration>
</execution> </execution>

View File

@ -19,8 +19,11 @@
package io.druid.examples.web; package io.druid.examples.web;
import com.google.api.client.repackaged.com.google.common.base.Throwables;
import com.google.common.base.Preconditions;
import com.google.common.io.InputSupplier; import com.google.common.io.InputSupplier;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import org.apache.commons.validator.routines.UrlValidator;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.IOException; import java.io.IOException;
@ -31,25 +34,25 @@ import java.net.URLConnection;
public class WebJsonSupplier implements InputSupplier<BufferedReader> public class WebJsonSupplier implements InputSupplier<BufferedReader>
{ {
private static final EmittingLogger log = new EmittingLogger(WebJsonSupplier.class); private static final EmittingLogger log = new EmittingLogger(WebJsonSupplier.class);
private static final UrlValidator urlValidator = new UrlValidator();
private String urlString;
private URL url; private URL url;
public WebJsonSupplier(String urlString) public WebJsonSupplier(String urlString)
{ {
this.urlString = urlString; Preconditions.checkState(urlValidator.isValid(urlString));
try { try {
this.url = new URL(urlString); this.url = new URL(urlString);
} }
catch (Exception e) { catch (Exception e) {
log.error(e,"Malformed url"); throw Throwables.propagate(e);
} }
} }
@Override @Override
public BufferedReader getInput() throws IOException public BufferedReader getInput() throws IOException
{ {
URL url = new URL(urlString);
URLConnection connection = url.openConnection(); URLConnection connection = url.openConnection();
connection.setDoInput(true); connection.setDoInput(true);
return new BufferedReader(new InputStreamReader(url.openStream())); return new BufferedReader(new InputStreamReader(url.openStream()));

View File

@ -22,15 +22,14 @@ package io.druid.examples.web;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.net.MalformedURLException;
public class WebJsonSupplierTest public class WebJsonSupplierTest
{ {
@Test(expected = IOException.class) @Test(expected = IllegalStateException.class)
public void checkInvalidUrl() throws Exception public void checkInvalidUrl() throws Exception
{ {
String invalidURL = "http://invalid.url."; String invalidURL = "http://invalid.url.";
WebJsonSupplier supplier = new WebJsonSupplier(invalidURL); WebJsonSupplier supplier = new WebJsonSupplier(invalidURL);
supplier.getInput();
} }
} }

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.70-SNAPSHOT</version> <version>0.6.102-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>
@ -52,7 +52,7 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId> <artifactId>hadoop-client</artifactId>
<scope>compile</scope> <scope>compile</scope>
</dependency> </dependency>
<dependency> <dependency>

View File

@ -66,12 +66,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
@Override @Override
public String getPathForHadoop(String dataSource) public String getPathForHadoop(String dataSource)
{ {
try { return new Path(config.getStorageDirectory(), dataSource).toUri().toString();
return new Path(config.getStorageDirectory(), dataSource).makeQualified(FileSystem.get(hadoopConfig)).toString();
}
catch (IOException e) {
throw Throwables.propagate(e);
}
} }
@Override @Override

View File

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

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.70-SNAPSHOT</version> <version>0.6.102-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>
@ -67,7 +67,7 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId> <artifactId>hadoop-client</artifactId>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.core</groupId> <groupId>com.fasterxml.jackson.core</groupId>

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import io.druid.db.DbConnector; import io.druid.db.DbConnector;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.IDBI;
@ -39,13 +40,15 @@ public class DbUpdaterJob implements Jobby
private final HadoopDruidIndexerConfig config; private final HadoopDruidIndexerConfig config;
private final IDBI dbi; private final IDBI dbi;
private final DbConnector dbConnector;
public DbUpdaterJob( public DbUpdaterJob(
HadoopDruidIndexerConfig config HadoopDruidIndexerConfig config
) )
{ {
this.config = config; this.config = config;
this.dbi = new DbConnector(config.getSchema().getIOConfig().getMetadataUpdateSpec(), null).getDBI(); this.dbConnector = new DbConnector(config.getSchema().getIOConfig().getMetadataUpdateSpec(), null);
this.dbi = this.dbConnector.getDBI();
} }
@Override @Override
@ -61,8 +64,11 @@ public class DbUpdaterJob implements Jobby
{ {
final PreparedBatch batch = handle.prepareBatch( final PreparedBatch batch = handle.prepareBatch(
String.format( String.format(
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " dbConnector.isPostgreSQL() ?
+ "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)" :
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable() config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable()
) )
); );
@ -75,7 +81,7 @@ public class DbUpdaterJob implements Jobby
.put("created_date", new DateTime().toString()) .put("created_date", new DateTime().toString())
.put("start", segment.getInterval().getStart().toString()) .put("start", segment.getInterval().getStart().toString())
.put("end", segment.getInterval().getEnd().toString()) .put("end", segment.getInterval().getEnd().toString())
.put("partitioned", segment.getShardSpec().getPartitionNum()) .put("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? 0 : 1)
.put("version", segment.getVersion()) .put("version", segment.getVersion())
.put("used", true) .put("used", true)
.put("payload", HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(segment)) .put("payload", HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(segment))
@ -94,5 +100,4 @@ public class DbUpdaterJob implements Jobby
return true; return true;
} }
} }

View File

@ -37,6 +37,7 @@ import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector; import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector;
import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.HashBasedNumberedShardSpec;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -45,6 +46,7 @@ import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Partitioner;
import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
@ -65,7 +67,6 @@ import java.util.Set;
*/ */
public class DetermineHashedPartitionsJob implements Jobby public class DetermineHashedPartitionsJob implements Jobby
{ {
private static final int MAX_SHARDS = 128;
private static final Logger log = new Logger(DetermineHashedPartitionsJob.class); private static final Logger log = new Logger(DetermineHashedPartitionsJob.class);
private final HadoopDruidIndexerConfig config; private final HadoopDruidIndexerConfig config;
@ -98,8 +99,11 @@ public class DetermineHashedPartitionsJob implements Jobby
groupByJob.setOutputKeyClass(NullWritable.class); groupByJob.setOutputKeyClass(NullWritable.class);
groupByJob.setOutputValueClass(NullWritable.class); groupByJob.setOutputValueClass(NullWritable.class);
groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class);
groupByJob.setPartitionerClass(DetermineHashedPartitionsPartitioner.class);
if (!config.getSegmentGranularIntervals().isPresent()) { if (!config.getSegmentGranularIntervals().isPresent()) {
groupByJob.setNumReduceTasks(1); groupByJob.setNumReduceTasks(1);
} else {
groupByJob.setNumReduceTasks(config.getSegmentGranularIntervals().get().size());
} }
JobHelper.setupClasspath(config, groupByJob); JobHelper.setupClasspath(config, groupByJob);
@ -124,9 +128,6 @@ public class DetermineHashedPartitionsJob implements Jobby
if (!config.getSegmentGranularIntervals().isPresent()) { if (!config.getSegmentGranularIntervals().isPresent()) {
final Path intervalInfoPath = config.makeIntervalInfoPath(); final Path intervalInfoPath = config.makeIntervalInfoPath();
fileSystem = intervalInfoPath.getFileSystem(groupByJob.getConfiguration()); fileSystem = intervalInfoPath.getFileSystem(groupByJob.getConfiguration());
if (!fileSystem.exists(intervalInfoPath)) {
throw new ISE("Path[%s] didn't exist!?", intervalInfoPath);
}
List<Interval> intervals = config.jsonMapper.readValue( List<Interval> intervals = config.jsonMapper.readValue(
Utils.openInputStream(groupByJob, intervalInfoPath), new TypeReference<List<Interval>>() Utils.openInputStream(groupByJob, intervalInfoPath), new TypeReference<List<Interval>>()
{ {
@ -151,37 +152,25 @@ public class DetermineHashedPartitionsJob implements Jobby
if (fileSystem == null) { if (fileSystem == null) {
fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration()); fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration());
} }
if (fileSystem.exists(partitionInfoPath)) { final Long cardinality = config.jsonMapper.readValue(
Long cardinality = config.jsonMapper.readValue( Utils.openInputStream(groupByJob, partitionInfoPath), new TypeReference<Long>()
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);
} }
);
final int numberOfShards = (int) Math.ceil((double) cardinality / config.getTargetPartitionSize());
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);
} }
config.setShardSpecs(shardSpecs); config.setShardSpecs(shardSpecs);
log.info( log.info(
@ -328,13 +317,6 @@ public class DetermineHashedPartitionsJob implements Jobby
} }
} }
private byte[] getDataBytes(BytesWritable writable)
{
byte[] rv = new byte[writable.getLength()];
System.arraycopy(writable.getBytes(), 0, rv, 0, writable.getLength());
return rv;
}
@Override @Override
public void run(Context context) public void run(Context context)
throws IOException, InterruptedException throws IOException, InterruptedException
@ -362,6 +344,50 @@ public class DetermineHashedPartitionsJob implements Jobby
} }
} }
} }
public static class DetermineHashedPartitionsPartitioner
extends Partitioner<LongWritable, BytesWritable> implements Configurable
{
private Configuration config;
private boolean determineIntervals;
private Map<LongWritable, Integer> reducerLookup;
@Override
public int getPartition(LongWritable interval, BytesWritable text, int numPartitions)
{
if (config.get("mapred.job.tracker").equals("local") || determineIntervals) {
return 0;
} else {
return reducerLookup.get(interval);
}
}
@Override
public Configuration getConf()
{
return config;
}
@Override
public void setConf(Configuration config)
{
this.config = config;
HadoopDruidIndexerConfig hadoopConfig = HadoopDruidIndexerConfig.fromConfiguration(config);
if (hadoopConfig.getSegmentGranularIntervals().isPresent()) {
determineIntervals = false;
int reducerNumber = 0;
ImmutableMap.Builder<LongWritable, Integer> builder = ImmutableMap.builder();
for (Interval interval : hadoopConfig.getSegmentGranularIntervals().get()) {
builder.put(new LongWritable(interval.getStartMillis()), reducerNumber++);
}
reducerLookup = builder.build();
} else {
determineIntervals = true;
}
}
}
} }

View File

@ -218,23 +218,20 @@ public class DeterminePartitionsJob implements Jobby
if (fileSystem == null) { if (fileSystem == null) {
fileSystem = partitionInfoPath.getFileSystem(dimSelectionJob.getConfiguration()); fileSystem = partitionInfoPath.getFileSystem(dimSelectionJob.getConfiguration());
} }
if (fileSystem.exists(partitionInfoPath)) { List<ShardSpec> specs = config.jsonMapper.readValue(
List<ShardSpec> specs = config.jsonMapper.readValue( Utils.openInputStream(dimSelectionJob, partitionInfoPath), new TypeReference<List<ShardSpec>>()
Utils.openInputStream(dimSelectionJob, partitionInfoPath), new TypeReference<List<ShardSpec>>() {
{
}
);
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]", segmentGranularity, i, actualSpecs.get(i));
}
shardSpecs.put(segmentGranularity.getStart(), actualSpecs);
} else {
log.info("Path[%s] didn't exist!?", partitionInfoPath);
} }
);
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]", segmentGranularity, i, actualSpecs.get(i));
}
shardSpecs.put(segmentGranularity.getStart(), actualSpecs);
} }
config.setShardSpecs(shardSpecs); config.setShardSpecs(shardSpecs);

View File

@ -23,6 +23,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import io.druid.timeline.partition.HashBasedNumberedShardSpec;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.DateTimeComparator; import org.joda.time.DateTimeComparator;
@ -56,13 +57,28 @@ public class HadoopDruidDetermineConfigurationJob implements Jobby
if (config.isDeterminingPartitions()) { if (config.isDeterminingPartitions()) {
jobs.add(config.getPartitionsSpec().getPartitionJob(config)); jobs.add(config.getPartitionsSpec().getPartitionJob(config));
} else { } else {
int shardsPerInterval = config.getPartitionsSpec().getNumShards();
Map<DateTime, List<HadoopyShardSpec>> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance()); Map<DateTime, List<HadoopyShardSpec>> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance());
int shardCount = 0; int shardCount = 0;
for (Interval segmentGranularity : config.getSegmentGranularIntervals().get()) { for (Interval segmentGranularity : config.getSegmentGranularIntervals().get()) {
DateTime bucket = segmentGranularity.getStart(); DateTime bucket = segmentGranularity.getStart();
final HadoopyShardSpec spec = new HadoopyShardSpec(new NoneShardSpec(), shardCount++); if (shardsPerInterval > 0) {
shardSpecs.put(bucket, Lists.newArrayList(spec)); List<HadoopyShardSpec> specs = Lists.newArrayListWithCapacity(shardsPerInterval);
log.info("DateTime[%s], spec[%s]", bucket, spec); for (int i = 0; i < shardsPerInterval; i++) {
specs.add(
new HadoopyShardSpec(
new HashBasedNumberedShardSpec(i, shardsPerInterval),
shardCount++
)
);
}
shardSpecs.put(bucket, specs);
log.info("DateTime[%s], spec[%s]", bucket, specs);
} else {
final HadoopyShardSpec spec = new HadoopyShardSpec(new NoneShardSpec(), shardCount++);
shardSpecs.put(bucket, Lists.newArrayList(spec));
log.info("DateTime[%s], spec[%s]", bucket, spec);
}
} }
config.setShardSpecs(shardSpecs); config.setShardSpecs(shardSpecs);
} }

View File

@ -20,6 +20,7 @@
package io.druid.indexer.partitions; package io.druid.indexer.partitions;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
public abstract class AbstractPartitionsSpec implements PartitionsSpec public abstract class AbstractPartitionsSpec implements PartitionsSpec
@ -30,11 +31,13 @@ public abstract class AbstractPartitionsSpec implements PartitionsSpec
private final long targetPartitionSize; private final long targetPartitionSize;
private final long maxPartitionSize; private final long maxPartitionSize;
private final boolean assumeGrouped; private final boolean assumeGrouped;
private final int numShards;
public AbstractPartitionsSpec( public AbstractPartitionsSpec(
Long targetPartitionSize, Long targetPartitionSize,
Long maxPartitionSize, Long maxPartitionSize,
Boolean assumeGrouped Boolean assumeGrouped,
Integer numShards
) )
{ {
this.targetPartitionSize = targetPartitionSize == null ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize; this.targetPartitionSize = targetPartitionSize == null ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize;
@ -42,6 +45,11 @@ public abstract class AbstractPartitionsSpec implements PartitionsSpec
? (long) (this.targetPartitionSize * DEFAULT_OVERSIZE_THRESHOLD) ? (long) (this.targetPartitionSize * DEFAULT_OVERSIZE_THRESHOLD)
: maxPartitionSize; : maxPartitionSize;
this.assumeGrouped = assumeGrouped == null ? false : assumeGrouped; this.assumeGrouped = assumeGrouped == null ? false : assumeGrouped;
this.numShards = numShards == null ? -1 : numShards;
Preconditions.checkArgument(
this.targetPartitionSize == -1 || this.numShards == -1,
"targetPartitionsSize and shardCount both cannot be set"
);
} }
@JsonProperty @JsonProperty
@ -67,4 +75,10 @@ public abstract class AbstractPartitionsSpec implements PartitionsSpec
{ {
return targetPartitionSize > 0; return targetPartitionSize > 0;
} }
@Override
public int getNumShards()
{
return numShards;
}
} }

View File

@ -31,17 +31,18 @@ public class HashedPartitionsSpec extends AbstractPartitionsSpec
{ {
public static HashedPartitionsSpec makeDefaultHashedPartitionsSpec() public static HashedPartitionsSpec makeDefaultHashedPartitionsSpec()
{ {
return new HashedPartitionsSpec(null, null, null); return new HashedPartitionsSpec(null, null, null, null);
} }
@JsonCreator @JsonCreator
public HashedPartitionsSpec( public HashedPartitionsSpec(
@JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize, @JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize,
@JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize, @JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize,
@JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped @JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped,
@JsonProperty("numShards") @Nullable Integer numShards
) )
{ {
super(targetPartitionSize, maxPartitionSize, assumeGrouped); super(targetPartitionSize, maxPartitionSize, assumeGrouped, numShards);
} }
@Override @Override

View File

@ -49,4 +49,7 @@ public interface PartitionsSpec
@JsonIgnore @JsonIgnore
public boolean isDeterminingPartitions(); public boolean isDeterminingPartitions();
@JsonProperty
public int getNumShards();
} }

View File

@ -21,9 +21,6 @@ package io.druid.indexer.partitions;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; 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; import javax.annotation.Nullable;
@ -35,9 +32,10 @@ public class RandomPartitionsSpec extends HashedPartitionsSpec
public RandomPartitionsSpec( public RandomPartitionsSpec(
@JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize, @JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize,
@JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize, @JsonProperty("maxPartitionSize") @Nullable Long maxPartitionSize,
@JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped @JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped,
@JsonProperty("numShards") @Nullable Integer numShards
) )
{ {
super(targetPartitionSize, maxPartitionSize, assumeGrouped); super(targetPartitionSize, maxPartitionSize, assumeGrouped, numShards);
} }
} }

View File

@ -41,7 +41,7 @@ public class SingleDimensionPartitionsSpec extends AbstractPartitionsSpec
@JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped @JsonProperty("assumeGrouped") @Nullable Boolean assumeGrouped
) )
{ {
super(targetPartitionSize, maxPartitionSize, assumeGrouped); super(targetPartitionSize, maxPartitionSize, assumeGrouped, null);
this.partitionDimension = partitionDimension; this.partitionDimension = partitionDimension;
} }

View File

@ -22,7 +22,6 @@ package io.druid.indexer;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables; import com.google.common.base.Throwables;
import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.partitions.RandomPartitionsSpec;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -37,6 +36,17 @@ public class HadoopDruidIndexerConfigTest
{ {
private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
public static <T> T jsonReadWriteRead(String s, Class<T> klass)
{
try {
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Test @Test
public void shouldMakeHDFSCompliantSegmentOutputPath() public void shouldMakeHDFSCompliantSegmentOutputPath()
{ {
@ -117,58 +127,4 @@ public class HadoopDruidIndexerConfigTest
); );
} }
private <T> T jsonReadWriteRead(String s, Class<T> klass)
{
try {
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
public void testRandomPartitionsSpec() throws Exception
{
{
final HadoopDruidIndexerConfig cfg;
try {
cfg = jsonReadWriteRead(
"{"
+ "\"partitionsSpec\":{"
+ " \"targetPartitionSize\":100,"
+ " \"type\":\"random\""
+ " }"
+ "}",
HadoopDruidIndexerConfig.class
);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
final PartitionsSpec partitionsSpec = cfg.getPartitionsSpec();
Assert.assertEquals(
"isDeterminingPartitions",
partitionsSpec.isDeterminingPartitions(),
true
);
Assert.assertEquals(
"getTargetPartitionSize",
partitionsSpec.getTargetPartitionSize(),
100
);
Assert.assertEquals(
"getMaxPartitionSize",
partitionsSpec.getMaxPartitionSize(),
150
);
Assert.assertTrue("partitionsSpec", partitionsSpec instanceof RandomPartitionsSpec);
}
}
} }

View File

@ -0,0 +1,117 @@
/*
* 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.google.common.base.Throwables;
import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.indexer.HadoopDruidIndexerConfigTest;
import org.junit.Assert;
import org.junit.Test;
/**
*/
public class HashedPartitionsSpecTest
{
@Test
public void testHashedPartitionsSpec() throws Exception
{
{
final PartitionsSpec partitionsSpec;
try {
partitionsSpec = HadoopDruidIndexerConfigTest.jsonReadWriteRead(
"{"
+ " \"targetPartitionSize\":100,"
+ " \"type\":\"hashed\""
+ "}",
PartitionsSpec.class
);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
Assert.assertEquals(
"isDeterminingPartitions",
partitionsSpec.isDeterminingPartitions(),
true
);
Assert.assertEquals(
"getTargetPartitionSize",
partitionsSpec.getTargetPartitionSize(),
100
);
Assert.assertEquals(
"getMaxPartitionSize",
partitionsSpec.getMaxPartitionSize(),
150
);
Assert.assertTrue("partitionsSpec", partitionsSpec instanceof HashedPartitionsSpec);
}
}
@Test
public void testHashedPartitionsSpecShardCount() throws Exception
{
final PartitionsSpec partitionsSpec;
try {
partitionsSpec = HadoopDruidIndexerConfigTest.jsonReadWriteRead(
"{"
+ " \"type\":\"hashed\","
+ " \"numShards\":2"
+ "}",
PartitionsSpec.class
);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
Assert.assertEquals(
"isDeterminingPartitions",
partitionsSpec.isDeterminingPartitions(),
false
);
Assert.assertEquals(
"getTargetPartitionSize",
partitionsSpec.getTargetPartitionSize(),
-1
);
Assert.assertEquals(
"getMaxPartitionSize",
partitionsSpec.getMaxPartitionSize(),
-1
);
Assert.assertEquals(
"shardCount",
partitionsSpec.getNumShards(),
2
);
Assert.assertTrue("partitionsSpec", partitionsSpec instanceof HashedPartitionsSpec);
}
}

View File

@ -0,0 +1,72 @@
/*
* 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.google.common.base.Throwables;
import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.indexer.HadoopDruidIndexerConfigTest;
import org.junit.Assert;
import org.junit.Test;
/**
*/
public class RandomPartitionsSpecTest
{
@Test
public void testRandomPartitionsSpec() throws Exception
{
{
final PartitionsSpec partitionsSpec;
try {
partitionsSpec = HadoopDruidIndexerConfigTest.jsonReadWriteRead(
"{"
+ " \"targetPartitionSize\":100,"
+ " \"type\":\"random\""
+ "}",
PartitionsSpec.class
);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
Assert.assertEquals(
"isDeterminingPartitions",
partitionsSpec.isDeterminingPartitions(),
true
);
Assert.assertEquals(
"getTargetPartitionSize",
partitionsSpec.getTargetPartitionSize(),
100
);
Assert.assertEquals(
"getMaxPartitionSize",
partitionsSpec.getMaxPartitionSize(),
150
);
Assert.assertTrue("partitionsSpec", partitionsSpec instanceof RandomPartitionsSpec);
}
}
}

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.70-SNAPSHOT</version> <version>0.6.102-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>
@ -71,7 +71,7 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId> <artifactId>hadoop-client</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
</dependencies> </dependencies>

View File

@ -64,7 +64,7 @@ public abstract class AbstractTask implements Task
this.id = Preconditions.checkNotNull(id, "id"); this.id = Preconditions.checkNotNull(id, "id");
this.groupId = Preconditions.checkNotNull(groupId, "groupId"); this.groupId = Preconditions.checkNotNull(groupId, "groupId");
this.taskResource = Preconditions.checkNotNull(taskResource, "resource"); this.taskResource = Preconditions.checkNotNull(taskResource, "resource");
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.dataSource = Preconditions.checkNotNull(dataSource.toLowerCase(), "dataSource");
} }
@JsonProperty @JsonProperty

View File

@ -65,11 +65,12 @@ public class HadoopIndexTask extends AbstractTask
extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class); extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class);
} }
private static String defaultHadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3"; public static String DEFAULT_HADOOP_COORDINATES = "org.apache.hadoop:hadoop-client:2.3.0";
@JsonIgnore @JsonIgnore
private final HadoopIngestionSchema schema; private final HadoopIngestionSchema schema;
@JsonIgnore @JsonIgnore
private final String hadoopCoordinates; private final List<String> hadoopDependencyCoordinates;
/** /**
* @param schema is used by the HadoopDruidIndexerJob to set up the appropriate parameters * @param schema is used by the HadoopDruidIndexerJob to set up the appropriate parameters
@ -85,7 +86,8 @@ public class HadoopIndexTask extends AbstractTask
public HadoopIndexTask( public HadoopIndexTask(
@JsonProperty("id") String id, @JsonProperty("id") String id,
@JsonProperty("schema") HadoopIngestionSchema schema, @JsonProperty("schema") HadoopIngestionSchema schema,
@JsonProperty("hadoopCoordinates") String hadoopCoordinates @JsonProperty("hadoopCoordinates") String hadoopCoordinates,
@JsonProperty("hadoopDependencyCoordinates") List<String> hadoopDependencyCoordinates
) )
{ {
super( super(
@ -102,7 +104,9 @@ public class HadoopIndexTask extends AbstractTask
Preconditions.checkArgument(schema.getIOConfig().getMetadataUpdateSpec() == null, "updaterJobSpec must be absent"); Preconditions.checkArgument(schema.getIOConfig().getMetadataUpdateSpec() == null, "updaterJobSpec must be absent");
this.schema = schema; this.schema = schema;
this.hadoopCoordinates = (hadoopCoordinates == null ? defaultHadoopCoordinates : hadoopCoordinates); this.hadoopDependencyCoordinates = hadoopDependencyCoordinates == null ? Arrays.<String>asList(
hadoopCoordinates == null ? DEFAULT_HADOOP_COORDINATES : hadoopCoordinates
) : hadoopDependencyCoordinates;
} }
@Override @Override
@ -134,20 +138,16 @@ public class HadoopIndexTask extends AbstractTask
} }
@JsonProperty @JsonProperty
public String getHadoopCoordinates() public List<String> getHadoopDependencyCoordinates()
{ {
return hadoopCoordinates; return hadoopDependencyCoordinates;
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public TaskStatus run(TaskToolbox toolbox) throws Exception public TaskStatus run(TaskToolbox toolbox) throws Exception
{ {
// setup Hadoop
final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig); final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig);
final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates(
aetherClient, hadoopCoordinates
);
final List<URL> extensionURLs = Lists.newArrayList(); final List<URL> extensionURLs = Lists.newArrayList();
for (String coordinate : extensionsConfig.getCoordinates()) { for (String coordinate : extensionsConfig.getCoordinates()) {
@ -163,7 +163,12 @@ public class HadoopIndexTask extends AbstractTask
final List<URL> driverURLs = Lists.newArrayList(); final List<URL> driverURLs = Lists.newArrayList();
driverURLs.addAll(nonHadoopURLs); driverURLs.addAll(nonHadoopURLs);
// put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts // put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts
driverURLs.addAll(Arrays.asList(((URLClassLoader) hadoopLoader).getURLs())); for (String hadoopDependencyCoordinate : hadoopDependencyCoordinates) {
final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates(
aetherClient, hadoopDependencyCoordinate
);
driverURLs.addAll(Arrays.asList(((URLClassLoader) hadoopLoader).getURLs()));
}
final URLClassLoader loader = new URLClassLoader(driverURLs.toArray(new URL[driverURLs.size()]), null); final URLClassLoader loader = new URLClassLoader(driverURLs.toArray(new URL[driverURLs.size()]), null);
Thread.currentThread().setContextClassLoader(loader); Thread.currentThread().setContextClassLoader(loader);

View File

@ -33,6 +33,7 @@ import io.druid.db.DbTablesConfig;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.TimelineObjectHolder;
import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.VersionedIntervalTimeline;
import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.skife.jdbi.v2.FoldController; import org.skife.jdbi.v2.FoldController;
@ -60,24 +61,24 @@ public class IndexerDBCoordinator
private final ObjectMapper jsonMapper; private final ObjectMapper jsonMapper;
private final DbTablesConfig dbTables; private final DbTablesConfig dbTables;
private final IDBI dbi; private final DbConnector dbConnector;
@Inject @Inject
public IndexerDBCoordinator( public IndexerDBCoordinator(
ObjectMapper jsonMapper, ObjectMapper jsonMapper,
DbTablesConfig dbTables, DbTablesConfig dbTables,
IDBI dbi DbConnector dbConnector
) )
{ {
this.jsonMapper = jsonMapper; this.jsonMapper = jsonMapper;
this.dbTables = dbTables; this.dbTables = dbTables;
this.dbi = dbi; this.dbConnector = dbConnector;
} }
public List<DataSegment> getUsedSegmentsForInterval(final String dataSource, final Interval interval) public List<DataSegment> getUsedSegmentsForInterval(final String dataSource, final Interval interval)
throws IOException throws IOException
{ {
final VersionedIntervalTimeline<String, DataSegment> timeline = dbi.withHandle( final VersionedIntervalTimeline<String, DataSegment> timeline = dbConnector.getDBI().withHandle(
new HandleCallback<VersionedIntervalTimeline<String, DataSegment>>() new HandleCallback<VersionedIntervalTimeline<String, DataSegment>>()
{ {
@Override @Override
@ -142,7 +143,7 @@ public class IndexerDBCoordinator
*/ */
public Set<DataSegment> announceHistoricalSegments(final Set<DataSegment> segments) throws IOException public Set<DataSegment> announceHistoricalSegments(final Set<DataSegment> segments) throws IOException
{ {
return dbi.inTransaction( return dbConnector.getDBI().inTransaction(
new TransactionCallback<Set<DataSegment>>() new TransactionCallback<Set<DataSegment>>()
{ {
@Override @Override
@ -180,7 +181,7 @@ public class IndexerDBCoordinator
try { try {
handle.createStatement( handle.createStatement(
String.format( String.format(
DbConnector.isPostgreSQL(handle) ? dbConnector.isPostgreSQL() ?
"INSERT INTO %s (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)": + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)":
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) "
@ -193,15 +194,13 @@ public class IndexerDBCoordinator
.bind("created_date", new DateTime().toString()) .bind("created_date", new DateTime().toString())
.bind("start", segment.getInterval().getStart().toString()) .bind("start", segment.getInterval().getStart().toString())
.bind("end", segment.getInterval().getEnd().toString()) .bind("end", segment.getInterval().getEnd().toString())
.bind("partitioned", segment.getShardSpec().getPartitionNum()) .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? 0 : 1)
.bind("version", segment.getVersion()) .bind("version", segment.getVersion())
.bind("used", true) .bind("used", true)
.bind("payload", jsonMapper.writeValueAsString(segment)) .bind("payload", jsonMapper.writeValueAsString(segment))
.execute(); .execute();
log.info("Published segment [%s] to DB", segment.getIdentifier()); log.info("Published segment [%s] to DB", segment.getIdentifier());
} catch(SQLException e) {
throw new IOException(e);
} catch(Exception e) { } catch(Exception e) {
if (e.getCause() instanceof SQLException && segmentExists(handle, segment)) { if (e.getCause() instanceof SQLException && segmentExists(handle, segment)) {
log.info("Found [%s] in DB, not updating DB", segment.getIdentifier()); log.info("Found [%s] in DB, not updating DB", segment.getIdentifier());
@ -234,7 +233,7 @@ public class IndexerDBCoordinator
public void updateSegmentMetadata(final Set<DataSegment> segments) throws IOException public void updateSegmentMetadata(final Set<DataSegment> segments) throws IOException
{ {
dbi.inTransaction( dbConnector.getDBI().inTransaction(
new TransactionCallback<Void>() new TransactionCallback<Void>()
{ {
@Override @Override
@ -252,7 +251,7 @@ public class IndexerDBCoordinator
public void deleteSegments(final Set<DataSegment> segments) throws IOException public void deleteSegments(final Set<DataSegment> segments) throws IOException
{ {
dbi.inTransaction( dbConnector.getDBI().inTransaction(
new TransactionCallback<Void>() new TransactionCallback<Void>()
{ {
@Override @Override
@ -295,7 +294,7 @@ public class IndexerDBCoordinator
public List<DataSegment> getUnusedSegmentsForInterval(final String dataSource, final Interval interval) public List<DataSegment> getUnusedSegmentsForInterval(final String dataSource, final Interval interval)
{ {
List<DataSegment> matchingSegments = dbi.withHandle( List<DataSegment> matchingSegments = dbConnector.getDBI().withHandle(
new HandleCallback<List<DataSegment>>() new HandleCallback<List<DataSegment>>()
{ {
@Override @Override
@ -303,7 +302,7 @@ public class IndexerDBCoordinator
{ {
return handle.createQuery( return handle.createQuery(
String.format( String.format(
DbConnector.isPostgreSQL(handle)? dbConnector.isPostgreSQL() ?
"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":
"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() dbTables.getSegmentsTable()

View File

@ -427,6 +427,7 @@ public class TaskSerdeTest
null, null,
null null
), ),
null,
null null
); );

View File

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

View File

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

57
pom.xml
View File

@ -23,14 +23,14 @@
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<packaging>pom</packaging> <packaging>pom</packaging>
<version>0.6.70-SNAPSHOT</version> <version>0.6.102-SNAPSHOT</version>
<name>druid</name> <name>druid</name>
<description>druid</description> <description>druid</description>
<scm> <scm>
<connection>scm:git:ssh://git@github.com/metamx/druid.git</connection> <connection>scm:git:ssh://git@github.com/metamx/druid.git</connection>
<developerConnection>scm:git:ssh://git@github.com/metamx/druid.git</developerConnection> <developerConnection>scm:git:ssh://git@github.com/metamx/druid.git</developerConnection>
<url>http://www.github.com/metamx/druid</url> <url>http://www.github.com/metamx/druid</url>
<tag>druid-0.6.69-SNAPSHOT</tag> <tag>druid-0.6.100-SNAPSHOT</tag>
</scm> </scm>
<prerequisites> <prerequisites>
@ -39,7 +39,7 @@
<properties> <properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding> <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.25.3</metamx.java-util.version> <metamx.java-util.version>0.25.4</metamx.java-util.version>
<apache.curator.version>2.4.0</apache.curator.version> <apache.curator.version>2.4.0</apache.curator.version>
<druid.api.version>0.2.0-SNAPSHOT</druid.api.version> <druid.api.version>0.2.0-SNAPSHOT</druid.api.version>
</properties> </properties>
@ -74,12 +74,12 @@
<dependency> <dependency>
<groupId>com.metamx</groupId> <groupId>com.metamx</groupId>
<artifactId>emitter</artifactId> <artifactId>emitter</artifactId>
<version>0.2.9</version> <version>0.2.11</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.metamx</groupId> <groupId>com.metamx</groupId>
<artifactId>http-client</artifactId> <artifactId>http-client</artifactId>
<version>0.8.5</version> <version>0.9.2</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.metamx</groupId> <groupId>com.metamx</groupId>
@ -94,7 +94,7 @@
<dependency> <dependency>
<groupId>com.metamx</groupId> <groupId>com.metamx</groupId>
<artifactId>server-metrics</artifactId> <artifactId>server-metrics</artifactId>
<version>0.0.5</version> <version>0.0.9</version>
</dependency> </dependency>
<dependency> <dependency>
@ -174,6 +174,12 @@
<groupId>org.apache.curator</groupId> <groupId>org.apache.curator</groupId>
<artifactId>curator-framework</artifactId> <artifactId>curator-framework</artifactId>
<version>${apache.curator.version}</version> <version>${apache.curator.version}</version>
<exclusions>
<exclusion>
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
</exclusions>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.curator</groupId> <groupId>org.apache.curator</groupId>
@ -218,37 +224,37 @@
<dependency> <dependency>
<groupId>com.fasterxml.jackson.core</groupId> <groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId> <artifactId>jackson-annotations</artifactId>
<version>2.3.1</version> <version>2.2.3</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.core</groupId> <groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId> <artifactId>jackson-core</artifactId>
<version>2.3.1</version> <version>2.2.3</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.core</groupId> <groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId> <artifactId>jackson-databind</artifactId>
<version>2.3.1</version> <version>2.2.3</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.datatype</groupId> <groupId>com.fasterxml.jackson.datatype</groupId>
<artifactId>jackson-datatype-guava</artifactId> <artifactId>jackson-datatype-guava</artifactId>
<version>2.3.1</version> <version>2.2.3</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.datatype</groupId> <groupId>com.fasterxml.jackson.datatype</groupId>
<artifactId>jackson-datatype-joda</artifactId> <artifactId>jackson-datatype-joda</artifactId>
<version>2.3.1</version> <version>2.2.3</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.dataformat</groupId> <groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-smile</artifactId> <artifactId>jackson-dataformat-smile</artifactId>
<version>2.3.1</version> <version>2.2.3</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.jaxrs</groupId> <groupId>com.fasterxml.jackson.jaxrs</groupId>
<artifactId>jackson-jaxrs-json-provider</artifactId> <artifactId>jackson-jaxrs-json-provider</artifactId>
<version>2.3.1</version> <version>2.2.3</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.codehaus.jackson</groupId> <groupId>org.codehaus.jackson</groupId>
@ -313,17 +319,17 @@
<dependency> <dependency>
<groupId>org.eclipse.jetty</groupId> <groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId> <artifactId>jetty-server</artifactId>
<version>8.1.11.v20130520</version> <version>9.1.4.v20140401</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.eclipse.jetty</groupId> <groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-servlet</artifactId> <artifactId>jetty-servlet</artifactId>
<version>8.1.11.v20130520</version> <version>9.1.4.v20140401</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.eclipse.jetty</groupId> <groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-servlets</artifactId> <artifactId>jetty-servlets</artifactId>
<version>8.1.11.v20130520</version> <version>9.1.4.v20140401</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>joda-time</groupId> <groupId>joda-time</groupId>
@ -373,7 +379,7 @@
<dependency> <dependency>
<groupId>com.google.protobuf</groupId> <groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId> <artifactId>protobuf-java</artifactId>
<version>2.4.0a</version> <version>2.5.0</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>io.tesla.aether</groupId> <groupId>io.tesla.aether</groupId>
@ -402,8 +408,8 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId> <artifactId>hadoop-client</artifactId>
<version>1.0.3</version> <version>2.3.0</version>
<scope>provided</scope> <scope>provided</scope>
</dependency> </dependency>
@ -535,6 +541,19 @@
<artifactId>maven-assembly-plugin</artifactId> <artifactId>maven-assembly-plugin</artifactId>
<version>2.4</version> <version>2.4</version>
</plugin> </plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-release-plugin</artifactId>
<version>2.4.2</version>
<dependencies>
<dependency>
<groupId>org.apache.maven.scm</groupId>
<artifactId>maven-scm-provider-gitexe</artifactId>
<!-- This version is necessary for use with git version 1.8.5 and above -->
<version>1.8.1</version>
</dependency>
</dependencies>
</plugin>
</plugins> </plugins>
</pluginManagement> </pluginManagement>
</build> </build>

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.70-SNAPSHOT</version> <version>0.6.102-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>
@ -91,6 +91,10 @@
<artifactId>easymock</artifactId> <artifactId>easymock</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>com.google.caliper</groupId>
<artifactId>caliper</artifactId>
</dependency>
</dependencies> </dependencies>
<build> <build>

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.metamx.common.ISE;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.Duration; import org.joda.time.Duration;
@ -37,14 +38,14 @@ public abstract class BaseQuery<T> implements Query<T>
{ {
public static String QUERYID = "queryId"; public static String QUERYID = "queryId";
private final DataSource dataSource; private final DataSource dataSource;
private final Map<String, String> context; private final Map<String, Object> context;
private final QuerySegmentSpec querySegmentSpec; private final QuerySegmentSpec querySegmentSpec;
private volatile Duration duration; private volatile Duration duration;
public BaseQuery( public BaseQuery(
DataSource dataSource, DataSource dataSource,
QuerySegmentSpec querySegmentSpec, QuerySegmentSpec querySegmentSpec,
Map<String, String> context Map<String, Object> context
) )
{ {
Preconditions.checkNotNull(dataSource, "dataSource can't be null"); Preconditions.checkNotNull(dataSource, "dataSource can't be null");
@ -102,28 +103,89 @@ public abstract class BaseQuery<T> implements Query<T>
} }
@JsonProperty @JsonProperty
public Map<String, String> getContext() public Map<String, Object> getContext()
{ {
return context; return context;
} }
@Override @Override
public String getContextValue(String key) public <ContextType> ContextType getContextValue(String key)
{ {
return context == null ? null : context.get(key); return context == null ? null : (ContextType) context.get(key);
} }
@Override @Override
public String getContextValue(String key, String defaultValue) public <ContextType> ContextType getContextValue(String key, ContextType defaultValue)
{ {
String retVal = getContextValue(key); ContextType retVal = getContextValue(key);
return retVal == null ? defaultValue : retVal; return retVal == null ? defaultValue : retVal;
} }
protected Map<String, String> computeOverridenContext(Map<String, String> overrides) @Override
public int getContextPriority(int defaultValue)
{ {
Map<String, String> overridden = Maps.newTreeMap(); if (context == null) {
final Map<String, String> context = getContext(); return defaultValue;
}
Object val = context.get("priority");
if (val == null) {
return defaultValue;
}
if (val instanceof String) {
return Integer.parseInt((String) val);
} else if (val instanceof Integer) {
return (int) val;
} else {
throw new ISE("Unknown type [%s]", val.getClass());
}
}
@Override
public boolean getContextBySegment(boolean defaultValue)
{
return parseBoolean("bySegment", defaultValue);
}
@Override
public boolean getContextPopulateCache(boolean defaultValue)
{
return parseBoolean("populateCache", defaultValue);
}
@Override
public boolean getContextUseCache(boolean defaultValue)
{
return parseBoolean("useCache", defaultValue);
}
@Override
public boolean getContextFinalize(boolean defaultValue)
{
return parseBoolean("finalize", defaultValue);
}
private boolean parseBoolean(String key, boolean defaultValue)
{
if (context == null) {
return defaultValue;
}
Object val = context.get(key);
if (val == null) {
return defaultValue;
}
if (val instanceof String) {
return Boolean.parseBoolean((String) val);
} else if (val instanceof Boolean) {
return (boolean) val;
} else {
throw new ISE("Unknown type [%s]. Cannot parse!", val.getClass());
}
}
protected Map<String, Object> computeOverridenContext(Map<String, Object> overrides)
{
Map<String, Object> overridden = Maps.newTreeMap();
final Map<String, Object> context = getContext();
if (context != null) { if (context != null) {
overridden.putAll(context); overridden.putAll(context);
} }
@ -135,28 +197,41 @@ public abstract class BaseQuery<T> implements Query<T>
@Override @Override
public String getId() public String getId()
{ {
return getContextValue(QUERYID); return (String) getContextValue(QUERYID);
} }
@Override @Override
public Query withId(String id) public Query withId(String id)
{ {
return withOverriddenContext(ImmutableMap.of(QUERYID, id)); return withOverriddenContext(ImmutableMap.<String, Object>of(QUERYID, id));
} }
@Override @Override
public boolean equals(Object o) public boolean equals(Object o)
{ {
if (this == o) return true; if (this == o) {
if (o == null || getClass() != o.getClass()) return false; return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
BaseQuery baseQuery = (BaseQuery) o; BaseQuery baseQuery = (BaseQuery) o;
if (context != null ? !context.equals(baseQuery.context) : baseQuery.context != null) return false; if (context != null ? !context.equals(baseQuery.context) : baseQuery.context != null) {
if (dataSource != null ? !dataSource.equals(baseQuery.dataSource) : baseQuery.dataSource != null) return false;
if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) return false;
if (querySegmentSpec != null ? !querySegmentSpec.equals(baseQuery.querySegmentSpec) : baseQuery.querySegmentSpec != null)
return false; return false;
}
if (dataSource != null ? !dataSource.equals(baseQuery.dataSource) : baseQuery.dataSource != null) {
return false;
}
if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) {
return false;
}
if (querySegmentSpec != null
? !querySegmentSpec.equals(baseQuery.querySegmentSpec)
: baseQuery.querySegmentSpec != null) {
return false;
}
return true; return true;
} }

View File

@ -53,7 +53,7 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query)
{ {
if (Boolean.parseBoolean(query.getContextValue("bySegment"))) { if (query.getContextBySegment(false)) {
final Sequence<T> baseSequence = base.run(query); final Sequence<T> baseSequence = base.run(query);
return new Sequence<T>() return new Sequence<T>()
{ {

View File

@ -64,10 +64,44 @@ public class BySegmentResultValueClass<T>
@Override @Override
public String toString() public String toString()
{ {
return "BySegmentTimeseriesResultValue{" + return "BySegmentResultValue{" +
"results=" + results + "results=" + results +
", segmentId='" + segmentId + '\'' + ", segmentId='" + segmentId + '\'' +
", interval='" + interval + '\'' + ", interval='" + interval + '\'' +
'}'; '}';
} }
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
BySegmentResultValueClass that = (BySegmentResultValueClass) o;
if (interval != null ? !interval.equals(that.interval) : that.interval != null) {
return false;
}
if (results != null ? !results.equals(that.results) : that.results != null) {
return false;
}
if (segmentId != null ? !segmentId.equals(that.segmentId) : that.segmentId != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = results != null ? results.hashCode() : 0;
result = 31 * result + (segmentId != null ? segmentId.hashCode() : 0);
result = 31 * result + (interval != null ? interval.hashCode() : 0);
return result;
}
} }

View File

@ -37,7 +37,7 @@ public abstract class BySegmentSkippingQueryRunner<T> implements QueryRunner<T>
@Override @Override
public Sequence<T> run(Query<T> query) public Sequence<T> run(Query<T> query)
{ {
if (Boolean.parseBoolean(query.getContextValue("bySegment"))) { if (query.getContextBySegment(false)) {
return baseRunner.run(query); return baseRunner.run(query);
} }

View File

@ -35,8 +35,10 @@ import com.metamx.common.logger.Logger;
import java.util.Arrays; import java.util.Arrays;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future; import java.util.concurrent.Future;
/** /**
@ -83,7 +85,7 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query)
{ {
final int priority = Integer.parseInt(query.getContextValue("priority", "0")); final int priority = query.getContextPriority(0);
return new BaseSequence<T, Iterator<T>>( return new BaseSequence<T, Iterator<T>>(
new BaseSequence.IteratorMaker<T, Iterator<T>>() new BaseSequence.IteratorMaker<T, Iterator<T>>()
@ -110,7 +112,18 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
if (input == null) { if (input == null) {
throw new ISE("Input is null?! How is this possible?!"); throw new ISE("Input is null?! How is this possible?!");
} }
return Sequences.toList(input.run(query), Lists.<T>newArrayList());
Sequence<T> result = input.run(query);
if (result == null) {
throw new ISE("Got a null result! Segments are missing!");
}
List<T> retVal = Sequences.toList(result, Lists.<T>newArrayList());
if (retVal == null) {
throw new ISE("Got a null list of results! WTF?!");
}
return retVal;
} }
catch (Exception e) { catch (Exception e) {
log.error(e, "Exception with one of the sequences!"); log.error(e, "Exception with one of the sequences!");

View File

@ -304,7 +304,7 @@ public class Druids
private QueryGranularity granularity; private QueryGranularity granularity;
private List<AggregatorFactory> aggregatorSpecs; private List<AggregatorFactory> aggregatorSpecs;
private List<PostAggregator> postAggregatorSpecs; private List<PostAggregator> postAggregatorSpecs;
private Map<String, String> context; private Map<String, Object> context;
private TimeseriesQueryBuilder() private TimeseriesQueryBuilder()
{ {
@ -384,7 +384,7 @@ public class Druids
return postAggregatorSpecs; return postAggregatorSpecs;
} }
public Map<String, String> getContext() public Map<String, Object> getContext()
{ {
return context; return context;
} }
@ -465,7 +465,7 @@ public class Druids
return this; return this;
} }
public TimeseriesQueryBuilder context(Map<String, String> c) public TimeseriesQueryBuilder context(Map<String, Object> c)
{ {
context = c; context = c;
return this; return this;
@ -505,7 +505,7 @@ public class Druids
private QuerySegmentSpec querySegmentSpec; private QuerySegmentSpec querySegmentSpec;
private List<String> dimensions; private List<String> dimensions;
private SearchQuerySpec querySpec; private SearchQuerySpec querySpec;
private Map<String, String> context; private Map<String, Object> context;
public SearchQueryBuilder() public SearchQueryBuilder()
{ {
@ -660,7 +660,7 @@ public class Druids
return this; return this;
} }
public SearchQueryBuilder context(Map<String, String> c) public SearchQueryBuilder context(Map<String, Object> c)
{ {
context = c; context = c;
return this; return this;
@ -690,7 +690,7 @@ public class Druids
{ {
private DataSource dataSource; private DataSource dataSource;
private QuerySegmentSpec querySegmentSpec; private QuerySegmentSpec querySegmentSpec;
private Map<String, String> context; private Map<String, Object> context;
public TimeBoundaryQueryBuilder() public TimeBoundaryQueryBuilder()
{ {
@ -746,7 +746,7 @@ public class Druids
return this; return this;
} }
public TimeBoundaryQueryBuilder context(Map<String, String> c) public TimeBoundaryQueryBuilder context(Map<String, Object> c)
{ {
context = c; context = c;
return this; return this;

View File

@ -24,7 +24,8 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.FinalizeMetricManipulationFn;
import io.druid.query.aggregation.IdentityMetricManipulationFn;
import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.MetricManipulationFn;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -48,62 +49,55 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query)
{ {
final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment")); final boolean isBySegment = query.getContextBySegment(false);
final boolean shouldFinalize = Boolean.parseBoolean(query.getContextValue("finalize", "true")); final boolean shouldFinalize = query.getContextFinalize(true);
final Query<T> queryToRun;
final Function<T, T> finalizerFn;
final MetricManipulationFn metricManipulationFn;
if (shouldFinalize) { if (shouldFinalize) {
Function<T, T> finalizerFn; queryToRun = query.withOverriddenContext(ImmutableMap.<String, Object>of("finalize", false));
if (isBySegment) { metricManipulationFn = new FinalizeMetricManipulationFn();
finalizerFn = new Function<T, T>()
{
final Function<T, T> baseFinalizer = toolChest.makeMetricManipulatorFn(
query,
new MetricManipulationFn()
{
@Override
public Object manipulate(AggregatorFactory factory, Object object)
{
return factory.finalizeComputation(factory.deserialize(object));
}
}
);
@Override } else {
@SuppressWarnings("unchecked") queryToRun = query;
public T apply(@Nullable T input) metricManipulationFn = new IdentityMetricManipulationFn();
{
Result<BySegmentResultValueClass<T>> result = (Result<BySegmentResultValueClass<T>>) input;
BySegmentResultValueClass<T> resultsClass = result.getValue();
return (T) new Result<BySegmentResultValueClass>(
result.getTimestamp(),
new BySegmentResultValueClass(
Lists.transform(resultsClass.getResults(), baseFinalizer),
resultsClass.getSegmentId(),
resultsClass.getInterval()
)
);
}
};
}
else {
finalizerFn = toolChest.makeMetricManipulatorFn(
query,
new MetricManipulationFn()
{
@Override
public Object manipulate(AggregatorFactory factory, Object object)
{
return factory.finalizeComputation(object);
}
}
);
}
return Sequences.map(
baseRunner.run(query.withOverriddenContext(ImmutableMap.of("finalize", "false"))),
finalizerFn
);
} }
return baseRunner.run(query); if (isBySegment) {
finalizerFn = new Function<T, T>()
{
final Function<T, T> baseFinalizer = toolChest.makePostComputeManipulatorFn(
query,
metricManipulationFn
);
@Override
@SuppressWarnings("unchecked")
public T apply(@Nullable T input)
{
Result<BySegmentResultValueClass<T>> result = (Result<BySegmentResultValueClass<T>>) input;
BySegmentResultValueClass<T> resultsClass = result.getValue();
return (T) new Result<BySegmentResultValueClass>(
result.getTimestamp(),
new BySegmentResultValueClass(
Lists.transform(resultsClass.getResults(), baseFinalizer),
resultsClass.getSegmentId(),
resultsClass.getInterval()
)
);
}
};
} else {
finalizerFn = toolChest.makePostComputeManipulatorFn(query, metricManipulationFn);
}
return Sequences.map(
baseRunner.run(queryToRun),
finalizerFn
);
} }
} }

View File

@ -83,7 +83,7 @@ public class GroupByParallelQueryRunner implements QueryRunner<Row>
query, query,
configSupplier.get() configSupplier.get()
); );
final int priority = Integer.parseInt(query.getContextValue("priority", "0")); final int priority = query.getContextPriority(0);
if (Iterables.isEmpty(queryables)) { if (Iterables.isEmpty(queryables)) {
log.warn("No queryables found."); log.warn("No queryables found.");

View File

@ -50,6 +50,10 @@ public class IntervalChunkingQueryRunner<T> implements QueryRunner<T>
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query)
{ {
if (period.getMillis() == 0) {
return baseRunner.run(query);
}
return Sequences.concat( return Sequences.concat(
FunctionalIterable FunctionalIterable
.create(query.getIntervals()) .create(query.getIntervals())

View File

@ -69,6 +69,11 @@ public class MetricsEmittingQueryRunner<T> implements QueryRunner<T>
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query)
{ {
final ServiceMetricEvent.Builder builder = builderFn.apply(query); final ServiceMetricEvent.Builder builder = builderFn.apply(query);
String queryId = query.getId();
if (queryId == null) {
queryId = "";
}
builder.setUser8(queryId);
return new Sequence<T>() return new Sequence<T>()
{ {

View File

@ -70,11 +70,18 @@ public interface Query<T>
public Duration getDuration(); public Duration getDuration();
public String getContextValue(String key); public <ContextType> ContextType getContextValue(String key);
public String getContextValue(String key, String defaultValue); public <ContextType> ContextType getContextValue(String key, ContextType defaultValue);
public Query<T> withOverriddenContext(Map<String, String> contextOverride); // For backwards compatibility
@Deprecated public int getContextPriority(int defaultValue);
@Deprecated public boolean getContextBySegment(boolean defaultValue);
@Deprecated public boolean getContextPopulateCache(boolean defaultValue);
@Deprecated public boolean getContextUseCache(boolean defaultValue);
@Deprecated public boolean getContextFinalize(boolean defaultValue);
public Query<T> withOverriddenContext(Map<String, Object> contextOverride);
public Query<T> withQuerySegmentSpec(QuerySegmentSpec spec); public Query<T> withQuerySegmentSpec(QuerySegmentSpec spec);

View File

@ -27,7 +27,7 @@ import org.joda.time.Period;
public class QueryConfig public class QueryConfig
{ {
@JsonProperty @JsonProperty
private Period chunkPeriod = Period.months(1); private Period chunkPeriod = new Period();
public Period getChunkPeriod() public Period getChunkPeriod()
{ {

View File

@ -44,8 +44,16 @@ public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultT
* @return * @return
*/ */
public abstract Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences); public abstract Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences);
public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query);
public abstract Function<ResultType, ResultType> makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn);
public abstract Function<ResultType, ResultType> makePreComputeManipulatorFn(QueryType query, MetricManipulationFn fn);
public Function<ResultType, ResultType> makePostComputeManipulatorFn(QueryType query, MetricManipulationFn fn)
{
return makePreComputeManipulatorFn(query, fn);
}
public abstract TypeReference<ResultType> getResultTypeReference(); public abstract TypeReference<ResultType> getResultTypeReference();
public <T> CacheStrategy<ResultType, T, QueryType> getCacheStrategy(QueryType query) { public <T> CacheStrategy<ResultType, T, QueryType> getCacheStrategy(QueryType query) {

View File

@ -0,0 +1,85 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.aggregation;
import com.google.common.collect.Lists;
import com.metamx.common.ISE;
import com.metamx.common.Pair;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
public class AggregatorUtil
{
/**
* returns the list of dependent postAggregators that should be calculated in order to calculate given postAgg
*
* @param postAggregatorList List of postAggregator, there is a restriction that the list should be in an order
* such that all the dependencies of any given aggregator should occur before that aggregator.
* See AggregatorUtilTest.testOutOfOrderPruneDependentPostAgg for example.
* @param postAggName name of the postAgg on which dependency is to be calculated
*/
public static List<PostAggregator> pruneDependentPostAgg(List<PostAggregator> postAggregatorList, String postAggName)
{
LinkedList<PostAggregator> rv = Lists.newLinkedList();
Set<String> deps = new HashSet<>();
deps.add(postAggName);
// Iterate backwards to find the last calculated aggregate and add dependent aggregator as we find dependencies in reverse order
for (PostAggregator agg : Lists.reverse(postAggregatorList)) {
if (deps.contains(agg.getName())) {
rv.addFirst(agg); // add to the beginning of List
deps.remove(agg.getName());
deps.addAll(agg.getDependentFields());
}
}
return rv;
}
public static Pair<List<AggregatorFactory>, List<PostAggregator>> condensedAggregators(
List<AggregatorFactory> aggList,
List<PostAggregator> postAggList,
String metric
)
{
List<PostAggregator> condensedPostAggs = AggregatorUtil.pruneDependentPostAgg(
postAggList,
metric
);
// calculate dependent aggregators for these postAgg
Set<String> dependencySet = new HashSet<>();
dependencySet.add(metric);
for (PostAggregator postAggregator : condensedPostAggs) {
dependencySet.addAll(postAggregator.getDependentFields());
}
List<AggregatorFactory> condensedAggs = Lists.newArrayList();
for (AggregatorFactory aggregatorSpec : aggList) {
if (dependencySet.contains(aggregatorSpec.getName())) {
condensedAggs.add(aggregatorSpec);
}
}
return new Pair(condensedAggs, condensedPostAggs);
}
}

View File

@ -88,6 +88,10 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory
@Override @Override
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
// handle "NaN" / "Infinity" values serialized as strings in JSON
if (object instanceof String) {
return Double.parseDouble((String) object);
}
return object; return object;
} }

View File

@ -0,0 +1,31 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.aggregation;
/**
*/
public class FinalizeMetricManipulationFn implements MetricManipulationFn
{
@Override
public Object manipulate(AggregatorFactory factory, Object object)
{
return factory.finalizeComputation(object);
}
}

View File

@ -0,0 +1,31 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.query.aggregation;
/**
*/
public class IdentityMetricManipulationFn implements MetricManipulationFn
{
@Override
public Object manipulate(AggregatorFactory factory, Object object)
{
return object;
}
}

View File

@ -139,6 +139,10 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
@Override @Override
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
// handle "NaN" / "Infinity" values serialized as strings in JSON
if (object instanceof String) {
return Double.parseDouble((String) object);
}
return object; return object;
} }

View File

@ -85,6 +85,7 @@ public class MaxAggregatorFactory implements AggregatorFactory
@Override @Override
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
// handle "NaN" / "Infinity" values serialized as strings in JSON
if (object instanceof String) { if (object instanceof String) {
return Double.parseDouble((String) object); return Double.parseDouble((String) object);
} }

View File

@ -85,6 +85,7 @@ public class MinAggregatorFactory implements AggregatorFactory
@Override @Override
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
// handle "NaN" / "Infinity" values serialized as strings in JSON
if (object instanceof String) { if (object instanceof String) {
return Double.parseDouble((String) object); return Double.parseDouble((String) object);
} }

View File

@ -37,12 +37,12 @@ public class HLLCV0 extends HyperLogLogCollector
private static final ByteBuffer defaultStorageBuffer = ByteBuffer.wrap(new byte[]{0, 0, 0}).asReadOnlyBuffer(); private static final ByteBuffer defaultStorageBuffer = ByteBuffer.wrap(new byte[]{0, 0, 0}).asReadOnlyBuffer();
public HLLCV0() protected HLLCV0()
{ {
super(defaultStorageBuffer); super(defaultStorageBuffer);
} }
public HLLCV0(ByteBuffer buffer) protected HLLCV0(ByteBuffer buffer)
{ {
super(buffer); super(buffer);
} }
@ -149,4 +149,4 @@ public class HLLCV0 extends HyperLogLogCollector
{ {
return buffer.position() + HEADER_NUM_BYTES; return buffer.position() + HEADER_NUM_BYTES;
} }
} }

View File

@ -44,12 +44,12 @@ public class HLLCV1 extends HyperLogLogCollector
private static final ByteBuffer defaultStorageBuffer = ByteBuffer.wrap(new byte[]{VERSION, 0, 0, 0, 0, 0, 0}) private static final ByteBuffer defaultStorageBuffer = ByteBuffer.wrap(new byte[]{VERSION, 0, 0, 0, 0, 0, 0})
.asReadOnlyBuffer(); .asReadOnlyBuffer();
public HLLCV1() protected HLLCV1()
{ {
super(defaultStorageBuffer); super(defaultStorageBuffer);
} }
public HLLCV1(ByteBuffer buffer) protected HLLCV1(ByteBuffer buffer)
{ {
super(buffer); super(buffer);
} }

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.primitives.UnsignedBytes; import com.google.common.primitives.UnsignedBytes;
import com.metamx.common.IAE; import com.metamx.common.IAE;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -57,7 +56,6 @@ public abstract class HyperLogLogCollector implements Comparable<HyperLogLogColl
public static final double HIGH_CORRECTION_THRESHOLD = TWO_TO_THE_SIXTY_FOUR / 30.0d; public static final double HIGH_CORRECTION_THRESHOLD = TWO_TO_THE_SIXTY_FOUR / 30.0d;
public static final double CORRECTION_PARAMETER = ALPHA * NUM_BUCKETS * NUM_BUCKETS; public static final double CORRECTION_PARAMETER = ALPHA * NUM_BUCKETS * NUM_BUCKETS;
private static final Logger log = new Logger(HyperLogLogCollector.class);
private static final int bucketMask = 0x7ff; private static final int bucketMask = 0x7ff;
private static final int minBytesRequired = 10; private static final int minBytesRequired = 10;
private static final int bitsPerBucket = 4; private static final int bitsPerBucket = 4;
@ -202,9 +200,9 @@ public abstract class HyperLogLogCollector implements Comparable<HyperLogLogColl
return buffer.remaining() != NUM_BYTES_FOR_BUCKETS; return buffer.remaining() != NUM_BYTES_FOR_BUCKETS;
} }
private volatile ByteBuffer storageBuffer; private ByteBuffer storageBuffer;
private volatile int initPosition; private int initPosition;
private volatile Double estimatedCardinality; private Double estimatedCardinality;
public HyperLogLogCollector(ByteBuffer byteBuffer) public HyperLogLogCollector(ByteBuffer byteBuffer)
{ {
@ -328,80 +326,73 @@ public abstract class HyperLogLogCollector implements Comparable<HyperLogLogColl
convertToMutableByteBuffer(); convertToMutableByteBuffer();
} }
estimatedCardinality = null;
if (getRegisterOffset() < other.getRegisterOffset()) {
// "Swap" the buffers so that we are folding into the one with the higher offset
ByteBuffer newStorage = ByteBuffer.allocate(other.storageBuffer.remaining());
newStorage.put(other.storageBuffer.asReadOnlyBuffer());
newStorage.clear();
other.storageBuffer = storageBuffer;
other.initPosition = initPosition;
storageBuffer = newStorage;
initPosition = 0;
}
ByteBuffer otherBuffer = other.storageBuffer.asReadOnlyBuffer();
byte otherOffset = other.getRegisterOffset();
if (storageBuffer.remaining() != getNumBytesForDenseStorage()) { if (storageBuffer.remaining() != getNumBytesForDenseStorage()) {
convertToDenseStorage(); convertToDenseStorage();
} }
estimatedCardinality = null;
if (getRegisterOffset() < other.getRegisterOffset()) {
// "Swap" the buffers so that we are folding into the one with the higher offset
final ByteBuffer tmpBuffer = ByteBuffer.allocate(storageBuffer.remaining());
tmpBuffer.put(storageBuffer.asReadOnlyBuffer());
tmpBuffer.clear();
storageBuffer.duplicate().put(other.storageBuffer.asReadOnlyBuffer());
other = HyperLogLogCollector.makeCollector(tmpBuffer);
}
final ByteBuffer otherBuffer = other.storageBuffer.asReadOnlyBuffer();
final byte otherOffset = other.getRegisterOffset();
byte myOffset = getRegisterOffset(); byte myOffset = getRegisterOffset();
short numNonZero = getNumNonZeroRegisters(); short numNonZero = getNumNonZeroRegisters();
int offsetDiff = myOffset - otherOffset; final int offsetDiff = myOffset - otherOffset;
if (offsetDiff < 0) { if (offsetDiff < 0) {
throw new ISE("offsetDiff[%d] < 0, shouldn't happen because of swap.", offsetDiff); throw new ISE("offsetDiff[%d] < 0, shouldn't happen because of swap.", offsetDiff);
} }
byte otherOverflowValue = other.getMaxOverflowValue(); add(other.getMaxOverflowRegister(), other.getMaxOverflowValue());
short otherOverflowRegister = other.getMaxOverflowRegister();
add(otherOverflowRegister, otherOverflowValue);
int myPayloadStart = getPayloadBytePosition(); final int myPayloadStart = getPayloadBytePosition();
otherBuffer.position(other.getPayloadBytePosition()); otherBuffer.position(other.getPayloadBytePosition());
if (isSparse(otherBuffer)) { if (isSparse(otherBuffer)) {
while (otherBuffer.hasRemaining()) { while(otherBuffer.hasRemaining()) {
short position = otherBuffer.getShort(); final int payloadStartPosition = otherBuffer.getShort() - other.getNumHeaderBytes();
int payloadStartPosition = position - other.getNumHeaderBytes();
numNonZero += mergeAndStoreByteRegister( numNonZero += mergeAndStoreByteRegister(
storageBuffer,
myPayloadStart + payloadStartPosition, myPayloadStart + payloadStartPosition,
offsetDiff, offsetDiff,
otherBuffer.get() otherBuffer.get()
); );
if (numNonZero == NUM_BUCKETS) { }
myOffset += 1; if (numNonZero == NUM_BUCKETS) {
numNonZero = decrementBuckets(); numNonZero = decrementBuckets();
setRegisterOffset(myOffset); setRegisterOffset(++myOffset);
setNumNonZeroRegisters(numNonZero); setNumNonZeroRegisters(numNonZero);
offsetDiff = myOffset - otherOffset;
}
} }
} else { // dense } else { // dense
int position = getPayloadBytePosition(); int position = getPayloadBytePosition();
while (otherBuffer.hasRemaining()) { while (otherBuffer.hasRemaining()) {
numNonZero += mergeAndStoreByteRegister( numNonZero += mergeAndStoreByteRegister(
storageBuffer,
position, position,
offsetDiff, offsetDiff,
otherBuffer.get() otherBuffer.get()
); );
if (numNonZero == NUM_BUCKETS) {
myOffset += 1;
numNonZero = decrementBuckets();
setRegisterOffset(myOffset);
setNumNonZeroRegisters(numNonZero);
offsetDiff = myOffset - otherOffset;
}
position++; position++;
} }
if (numNonZero == NUM_BUCKETS) {
numNonZero = decrementBuckets();
setRegisterOffset(++myOffset);
setNumNonZeroRegisters(numNonZero);
}
} }
setRegisterOffset(myOffset); // no need to call setRegisterOffset(myOffset) here, since it gets updated every time myOffset is incremented
setNumNonZeroRegisters(numNonZero); setNumNonZeroRegisters(numNonZero);
return this; return this;
@ -531,15 +522,15 @@ public abstract class HyperLogLogCollector implements Comparable<HyperLogLogColl
private short decrementBuckets() private short decrementBuckets()
{ {
final int startPosition = getPayloadBytePosition();
short count = 0; short count = 0;
int startPosition = getPayloadBytePosition();
for (int i = startPosition; i < startPosition + NUM_BYTES_FOR_BUCKETS; i++) { for (int i = startPosition; i < startPosition + NUM_BYTES_FOR_BUCKETS; i++) {
byte val = (byte) (storageBuffer.get(i) - 0x11); final byte val = (byte) (storageBuffer.get(i) - 0x11);
if ((val & 0xf0) != 0) { if ((val & 0xf0) != 0) {
count++; ++count;
} }
if ((val & 0x0f) != 0) { if ((val & 0x0f) != 0) {
count++; ++count;
} }
storageBuffer.put(i, val); storageBuffer.put(i, val);
} }
@ -557,7 +548,7 @@ public abstract class HyperLogLogCollector implements Comparable<HyperLogLogColl
private void convertToDenseStorage() private void convertToDenseStorage()
{ {
ByteBuffer tmpBuffer = ByteBuffer.wrap(new byte[getNumBytesForDenseStorage()]); ByteBuffer tmpBuffer = ByteBuffer.allocate(getNumBytesForDenseStorage());
// put header // put header
setVersion(tmpBuffer); setVersion(tmpBuffer);
setRegisterOffset(tmpBuffer, getRegisterOffset()); setRegisterOffset(tmpBuffer, getRegisterOffset());
@ -614,39 +605,39 @@ public abstract class HyperLogLogCollector implements Comparable<HyperLogLogColl
* *
* @return * @return
*/ */
private int mergeAndStoreByteRegister( private static int mergeAndStoreByteRegister(
int position, final ByteBuffer storageBuffer,
int offsetDiff, final int position,
byte byteToAdd final int offsetDiff,
final byte byteToAdd
) )
{ {
if (byteToAdd == 0) { if (byteToAdd == 0) {
return 0; return 0;
} }
byte currVal = storageBuffer.get(position); final byte currVal = storageBuffer.get(position);
int upperNibble = currVal & 0xf0; final int upperNibble = currVal & 0xf0;
int lowerNibble = currVal & 0x0f; final int lowerNibble = currVal & 0x0f;
// subtract the differences so that the nibbles align // subtract the differences so that the nibbles align
int otherUpper = (byteToAdd & 0xf0) - (offsetDiff << bitsPerBucket); final int otherUpper = (byteToAdd & 0xf0) - (offsetDiff << bitsPerBucket);
int otherLower = (byteToAdd & 0x0f) - offsetDiff; final int otherLower = (byteToAdd & 0x0f) - offsetDiff;
final int newUpper = Math.max(upperNibble, otherUpper); final int newUpper = Math.max(upperNibble, otherUpper);
final int newLower = Math.max(lowerNibble, otherLower); final int newLower = Math.max(lowerNibble, otherLower);
storageBuffer.put(position, (byte) ((newUpper | newLower) & 0xff));
int numNoLongerZero = 0; int numNoLongerZero = 0;
if (upperNibble == 0 && newUpper > 0) { if (upperNibble == 0 && newUpper > 0) {
++numNoLongerZero; ++numNoLongerZero;
} }
if (lowerNibble == 0 && newLower > 0) { if (lowerNibble == 0 && newLower > 0) {
++numNoLongerZero; ++numNoLongerZero;
} }
storageBuffer.put(position, (byte) ((newUpper | newLower) & 0xff));
return numNoLongerZero; return numNoLongerZero;
} }

View File

@ -66,10 +66,13 @@ public class HyperUniquesBufferAggregator implements BufferAggregator
@Override @Override
public Object get(ByteBuffer buf, int position) public Object get(ByteBuffer buf, int position)
{ {
ByteBuffer dataCopyBuffer = ByteBuffer.allocate(HyperLogLogCollector.getLatestNumBytesForDenseStorage()); final int size = HyperLogLogCollector.getLatestNumBytesForDenseStorage();
ByteBuffer dataCopyBuffer = ByteBuffer.allocate(size);
ByteBuffer mutationBuffer = buf.duplicate(); ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position); mutationBuffer.position(position);
mutationBuffer.get(dataCopyBuffer.array()); mutationBuffer.limit(position + size);
dataCopyBuffer.put(mutationBuffer);
dataCopyBuffer.rewind();
return HyperLogLogCollector.makeCollector(dataCopyBuffer); return HyperLogLogCollector.makeCollector(dataCopyBuffer);
} }

View File

@ -79,17 +79,23 @@ public class HyperUniquesSerde extends ComplexMetricSerde
@Override @Override
public HyperLogLogCollector extractValue(InputRow inputRow, String metricName) public HyperLogLogCollector extractValue(InputRow inputRow, String metricName)
{ {
HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); Object rawValue = inputRow.getRaw(metricName);
List<String> dimValues = inputRow.getDimension(metricName); if (rawValue instanceof HyperLogLogCollector) {
if (dimValues == null) { return (HyperLogLogCollector) inputRow.getRaw(metricName);
} else {
HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector();
List<String> dimValues = inputRow.getDimension(metricName);
if (dimValues == null) {
return collector;
}
for (String dimensionValue : dimValues) {
collector.add(hashFn.hashBytes(dimensionValue.getBytes(Charsets.UTF_8)).asBytes());
}
return collector; return collector;
} }
for (String dimensionValue : dimValues) {
collector.add(hashFn.hashBytes(dimensionValue.getBytes(Charsets.UTF_8)).asBytes());
}
return collector;
} }
}; };
} }
@ -121,9 +127,7 @@ public class HyperUniquesSerde extends ComplexMetricSerde
public HyperLogLogCollector fromByteBuffer(ByteBuffer buffer, int numBytes) public HyperLogLogCollector fromByteBuffer(ByteBuffer buffer, int numBytes)
{ {
buffer.limit(buffer.position() + numBytes); buffer.limit(buffer.position() + numBytes);
return HyperLogLogCollector.makeCollector(buffer);
int remaining = buffer.remaining();
return (remaining % 3 == 0 || remaining == 1027) ? new HLLCV0(buffer) : new HLLCV1(buffer);
} }
@Override @Override

View File

@ -21,6 +21,7 @@ package io.druid.query.aggregation.post;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.PostAggregator;
@ -38,11 +39,13 @@ public class ConstantPostAggregator implements PostAggregator
@JsonCreator @JsonCreator
public ConstantPostAggregator( public ConstantPostAggregator(
@JsonProperty("name") String name, @JsonProperty("name") String name,
@JsonProperty("value") Number constantValue @JsonProperty("value") Number constantValue,
@JsonProperty("constantValue") Number backwardsCompatibleValue
) )
{ {
this.name = name; this.name = name;
this.constantValue = constantValue; this.constantValue = constantValue == null ? backwardsCompatibleValue : constantValue;
Preconditions.checkNotNull(this.constantValue);
} }
@Override @Override
@ -126,4 +129,5 @@ public class ConstantPostAggregator implements PostAggregator
result = 31 * result + (constantValue != null ? constantValue.hashCode() : 0); result = 31 * result + (constantValue != null ? constantValue.hashCode() : 0);
return result; return result;
} }
} }

View File

@ -67,4 +67,13 @@ public class JavaScriptDimFilter implements DimFilter
.put(functionBytes) .put(functionBytes)
.array(); .array();
} }
@Override
public String toString()
{
return "JavaScriptDimFilter{" +
"dimension='" + dimension + '\'' +
", function='" + function + '\'' +
'}';
}
} }

View File

@ -69,4 +69,13 @@ public class RegexDimFilter implements DimFilter
.put(patternBytes) .put(patternBytes)
.array(); .array();
} }
@Override
public String toString()
{
return "RegexDimFilter{" +
"dimension='" + dimension + '\'' +
", pattern='" + pattern + '\'' +
'}';
}
} }

View File

@ -64,9 +64,18 @@ public class SearchQueryDimFilter implements DimFilter
final byte[] queryBytes = query.getCacheKey(); final byte[] queryBytes = query.getCacheKey();
return ByteBuffer.allocate(1 + dimensionBytes.length + queryBytes.length) return ByteBuffer.allocate(1 + dimensionBytes.length + queryBytes.length)
.put(DimFilterCacheHelper.SEARCH_QUERY_TYPE_ID) .put(DimFilterCacheHelper.SEARCH_QUERY_TYPE_ID)
.put(dimensionBytes) .put(dimensionBytes)
.put(queryBytes) .put(queryBytes)
.array(); .array();
}
@Override
public String toString()
{
return "SearchQueryDimFilter{" +
"dimension='" + dimension + '\'' +
", query=" + query +
'}';
} }
} }

View File

@ -99,4 +99,13 @@ public class SpatialDimFilter implements DimFilter
result = 31 * result + (bound != null ? bound.hashCode() : 0); result = 31 * result + (bound != null ? bound.hashCode() : 0);
return result; return result;
} }
@Override
public String toString()
{
return "SpatialDimFilter{" +
"dimension='" + dimension + '\'' +
", bound=" + bound +
'}';
}
} }

View File

@ -86,7 +86,7 @@ public class GroupByQuery extends BaseQuery<Row>
@JsonProperty("having") HavingSpec havingSpec, @JsonProperty("having") HavingSpec havingSpec,
@JsonProperty("limitSpec") LimitSpec limitSpec, @JsonProperty("limitSpec") LimitSpec limitSpec,
@JsonProperty("orderBy") LimitSpec orderBySpec, @JsonProperty("orderBy") LimitSpec orderBySpec,
@JsonProperty("context") Map<String, String> context @JsonProperty("context") Map<String, Object> context
) )
{ {
super(dataSource, querySegmentSpec, context); super(dataSource, querySegmentSpec, context);
@ -147,7 +147,7 @@ public class GroupByQuery extends BaseQuery<Row>
HavingSpec havingSpec, HavingSpec havingSpec,
LimitSpec orderBySpec, LimitSpec orderBySpec,
Function<Sequence<Row>, Sequence<Row>> orderByLimitFn, Function<Sequence<Row>, Sequence<Row>> orderByLimitFn,
Map<String, String> context Map<String, Object> context
) )
{ {
super(dataSource, querySegmentSpec, context); super(dataSource, querySegmentSpec, context);
@ -222,7 +222,7 @@ public class GroupByQuery extends BaseQuery<Row>
} }
@Override @Override
public GroupByQuery withOverriddenContext(Map<String, String> contextOverride) public GroupByQuery withOverriddenContext(Map<String, Object> contextOverride)
{ {
return new GroupByQuery( return new GroupByQuery(
getDataSource(), getDataSource(),
@ -268,7 +268,7 @@ public class GroupByQuery extends BaseQuery<Row>
private List<PostAggregator> postAggregatorSpecs; private List<PostAggregator> postAggregatorSpecs;
private HavingSpec havingSpec; private HavingSpec havingSpec;
private Map<String, String> context; private Map<String, Object> context;
private LimitSpec limitSpec = null; private LimitSpec limitSpec = null;
private List<OrderByColumnSpec> orderByColumnSpecs = Lists.newArrayList(); private List<OrderByColumnSpec> orderByColumnSpecs = Lists.newArrayList();
@ -443,7 +443,7 @@ public class GroupByQuery extends BaseQuery<Row>
return this; return this;
} }
public Builder setContext(Map<String, String> context) public Builder setContext(Map<String, Object> context)
{ {
this.context = context; this.context = context;
return this; return this;

View File

@ -44,6 +44,7 @@ import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.BufferAggregator;
import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DimensionSpec; import io.druid.query.dimension.DimensionSpec;
import io.druid.query.extraction.DimExtractionFn;
import io.druid.segment.Cursor; import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector; import io.druid.segment.DimensionSelector;
import io.druid.segment.StorageAdapter; import io.druid.segment.StorageAdapter;
@ -69,7 +70,7 @@ public class GroupByQueryEngine
private final StupidPool<ByteBuffer> intermediateResultsBufferPool; private final StupidPool<ByteBuffer> intermediateResultsBufferPool;
@Inject @Inject
public GroupByQueryEngine ( public GroupByQueryEngine(
Supplier<GroupByQueryConfig> config, Supplier<GroupByQueryConfig> config,
@Global StupidPool<ByteBuffer> intermediateResultsBufferPool @Global StupidPool<ByteBuffer> intermediateResultsBufferPool
) )
@ -80,6 +81,12 @@ public class GroupByQueryEngine
public Sequence<Row> process(final GroupByQuery query, StorageAdapter storageAdapter) public Sequence<Row> process(final GroupByQuery query, StorageAdapter storageAdapter)
{ {
if (storageAdapter == null) {
throw new ISE(
"Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped."
);
}
final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals(); final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
if (intervals.size() != 1) { if (intervals.size() != 1) {
throw new IAE("Should only have one interval, got[%s]", intervals); throw new IAE("Should only have one interval, got[%s]", intervals);
@ -182,12 +189,11 @@ public class GroupByQueryEngine
final DimensionSelector dimSelector = dims.get(0); final DimensionSelector dimSelector = dims.get(0);
final IndexedInts row = dimSelector.getRow(); final IndexedInts row = dimSelector.getRow();
if (row.size() == 0) { if (row == null || row.size() == 0) {
ByteBuffer newKey = key.duplicate(); ByteBuffer newKey = key.duplicate();
newKey.putInt(dimSelector.getValueCardinality()); newKey.putInt(dimSelector.getValueCardinality());
unaggregatedBuffers = updateValues(newKey, dims.subList(1, dims.size())); unaggregatedBuffers = updateValues(newKey, dims.subList(1, dims.size()));
} } else {
else {
for (Integer dimValue : row) { for (Integer dimValue : row) {
ByteBuffer newKey = key.duplicate(); ByteBuffer newKey = key.duplicate();
newKey.putInt(dimValue); newKey.putInt(dimValue);
@ -201,8 +207,7 @@ public class GroupByQueryEngine
retVal.addAll(unaggregatedBuffers); retVal.addAll(unaggregatedBuffers);
} }
return retVal; return retVal;
} } else {
else {
key.clear(); key.clear();
Integer position = positions.get(key); Integer position = positions.get(key);
int[] increments = positionMaintainer.getIncrements(); int[] increments = positionMaintainer.getIncrements();
@ -266,8 +271,7 @@ public class GroupByQueryEngine
{ {
if (nextVal > max) { if (nextVal > max) {
return null; return null;
} } else {
else {
int retVal = (int) nextVal; int retVal = (int) nextVal;
nextVal += increment; nextVal += increment;
return retVal; return retVal;
@ -398,9 +402,14 @@ public class GroupByQueryEngine
ByteBuffer keyBuffer = input.getKey().duplicate(); ByteBuffer keyBuffer = input.getKey().duplicate();
for (int i = 0; i < dimensions.size(); ++i) { for (int i = 0; i < dimensions.size(); ++i) {
final DimensionSelector dimSelector = dimensions.get(i); final DimensionSelector dimSelector = dimensions.get(i);
final DimExtractionFn fn = dimensionSpecs.get(i).getDimExtractionFn();
final int dimVal = keyBuffer.getInt(); final int dimVal = keyBuffer.getInt();
if (dimSelector.getValueCardinality() != dimVal) { if (dimSelector.getValueCardinality() != dimVal) {
theEvent.put(dimNames.get(i), dimSelector.lookupName(dimVal)); if (fn != null) {
theEvent.put(dimNames.get(i), fn.apply(dimSelector.lookupName(dimVal)));
} else {
theEvent.put(dimNames.get(i), dimSelector.lookupName(dimVal));
}
} }
} }
@ -428,9 +437,10 @@ public class GroupByQueryEngine
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
public void close() { public void close()
{
// cleanup // cleanup
for(BufferAggregator agg : aggregators) { for (BufferAggregator agg : aggregators) {
agg.close(); agg.close();
} }
} }

View File

@ -58,7 +58,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
{ {
}; };
private static final String GROUP_BY_MERGE_KEY = "groupByMerge"; private static final String GROUP_BY_MERGE_KEY = "groupByMerge";
private static final Map<String, String> NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false"); private static final Map<String, Object> NO_MERGE_CONTEXT = ImmutableMap.<String, Object>of(GROUP_BY_MERGE_KEY, "false");
private final Supplier<GroupByQueryConfig> configSupplier; private final Supplier<GroupByQueryConfig> configSupplier;
private GroupByQueryEngine engine; // For running the outer query around a subquery private GroupByQueryEngine engine; // For running the outer query around a subquery
@ -80,7 +80,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
@Override @Override
public Sequence<Row> run(Query<Row> input) public Sequence<Row> run(Query<Row> input)
{ {
if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { if (Boolean.valueOf((String) input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) {
return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner); return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner);
} else { } else {
return runner.run(input); return runner.run(input);
@ -169,12 +169,11 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
.setUser5(Joiner.on(",").join(query.getIntervals())) .setUser5(Joiner.on(",").join(query.getIntervals()))
.setUser6(String.valueOf(query.hasFilters())) .setUser6(String.valueOf(query.hasFilters()))
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size())) .setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()))
.setUser9(Minutes.minutes(numMinutes).toString()) .setUser9(Minutes.minutes(numMinutes).toString());
.setUser10(query.getId());
} }
@Override @Override
public Function<Row, Row> makeMetricManipulatorFn(final GroupByQuery query, final MetricManipulationFn fn) public Function<Row, Row> makePreComputeManipulatorFn(final GroupByQuery query, final MetricManipulationFn fn)
{ {
return new Function<Row, Row>() return new Function<Row, Row>()
{ {

View File

@ -151,12 +151,11 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
.setUser4(query.getType()) .setUser4(query.getType())
.setUser5(Joiner.on(",").join(query.getIntervals())) .setUser5(Joiner.on(",").join(query.getIntervals()))
.setUser6(String.valueOf(query.hasFilters())) .setUser6(String.valueOf(query.hasFilters()))
.setUser9(Minutes.minutes(numMinutes).toString()) .setUser9(Minutes.minutes(numMinutes).toString());
.setUser10(query.getId());
} }
@Override @Override
public Function<SegmentAnalysis, SegmentAnalysis> makeMetricManipulatorFn( public Function<SegmentAnalysis, SegmentAnalysis> makePreComputeManipulatorFn(
SegmentMetadataQuery query, MetricManipulationFn fn SegmentMetadataQuery query, MetricManipulationFn fn
) )
{ {

View File

@ -104,47 +104,47 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
) )
{ {
return new ConcatQueryRunner<SegmentAnalysis>( return new ConcatQueryRunner<SegmentAnalysis>(
Sequences.map( Sequences.map(
Sequences.simple(queryRunners), Sequences.simple(queryRunners),
new Function<QueryRunner<SegmentAnalysis>, QueryRunner<SegmentAnalysis>>() new Function<QueryRunner<SegmentAnalysis>, QueryRunner<SegmentAnalysis>>()
{
@Override
public QueryRunner<SegmentAnalysis> apply(final QueryRunner<SegmentAnalysis> input)
{
return new QueryRunner<SegmentAnalysis>()
{ {
@Override @Override
public QueryRunner<SegmentAnalysis> apply(final QueryRunner<SegmentAnalysis> input) public Sequence<SegmentAnalysis> run(final Query<SegmentAnalysis> query)
{ {
return new QueryRunner<SegmentAnalysis>()
{
@Override
public Sequence<SegmentAnalysis> run(final Query<SegmentAnalysis> query)
{
Future<Sequence<SegmentAnalysis>> future = queryExecutor.submit( Future<Sequence<SegmentAnalysis>> future = queryExecutor.submit(
new Callable<Sequence<SegmentAnalysis>>() new Callable<Sequence<SegmentAnalysis>>()
{ {
@Override @Override
public Sequence<SegmentAnalysis> call() throws Exception public Sequence<SegmentAnalysis> call() throws Exception
{ {
return new ExecutorExecutingSequence<SegmentAnalysis>( return new ExecutorExecutingSequence<SegmentAnalysis>(
input.run(query), input.run(query),
queryExecutor queryExecutor
); );
} }
}
);
try {
return future.get();
} }
catch (InterruptedException e) { );
throw Throwables.propagate(e); try {
} return future.get();
catch (ExecutionException e) { }
throw Throwables.propagate(e); catch (InterruptedException e) {
} throw Throwables.propagate(e);
} }
}; catch (ExecutionException e) {
throw Throwables.propagate(e);
}
} }
} };
) }
); }
)
);
} }
@Override @Override

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