From 5472c4a249c4419a66124057769077d5e416e223 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 22 Mar 2016 14:16:34 -0700 Subject: [PATCH 1/3] Improved docs for multi-value dimensions. - Add central doc for multi-value dimensions, with some content from other docs. - Link to multi-value dimension doc from topN and groupBy docs. - Fixes a broken link from dimensionspecs.md, which was presciently already linking to this nonexistent doc. - Resolve inconsistent naming in docs & code (sometimes "multi-valued", sometimes "multi-value") in favor of "multi-value". (Backport of #2701) --- docs/content/design/segments.md | 2 +- docs/content/querying/dimensionspecs.md | 12 ++--- docs/content/querying/groupbyquery.md | 11 +++++ ...mizations.md => multi-value-dimensions.md} | 45 ++++++++++++------- docs/content/querying/topnquery.md | 13 ++++++ docs/content/toc.md | 2 +- .../druid/indexer/DeterminePartitionsJob.java | 2 +- .../segment/QueryableIndexStorageAdapter.java | 2 +- 8 files changed, 64 insertions(+), 25 deletions(-) rename docs/content/querying/{optimizations.md => multi-value-dimensions.md} (59%) diff --git a/docs/content/design/segments.md b/docs/content/design/segments.md index 7e03fd66d8cd..7d0b1258ae43 100644 --- a/docs/content/design/segments.md +++ b/docs/content/design/segments.md @@ -163,7 +163,7 @@ Each column is stored as two parts: 1. A Jackson-serialized ColumnDescriptor 2. The rest of the binary for the column -A ColumnDescriptor is essentially an object that allows us to use jackson’s polymorphic deserialization to add new and interesting methods of serialization with minimal impact to the code. It consists of some metadata about the column (what type is it, is it multi-valued, etc.) and then a list of serde logic that can deserialize the rest of the binary. +A ColumnDescriptor is essentially an object that allows us to use jackson’s polymorphic deserialization to add new and interesting methods of serialization with minimal impact to the code. It consists of some metadata about the column (what type is it, is it multi-value, etc.) and then a list of serde logic that can deserialize the rest of the binary. Sharding Data to Create Segments -------------------------------- diff --git a/docs/content/querying/dimensionspecs.md b/docs/content/querying/dimensionspecs.md index a765715f435c..ab6185daa851 100644 --- a/docs/content/querying/dimensionspecs.md +++ b/docs/content/querying/dimensionspecs.md @@ -351,14 +351,14 @@ Returns the dimension value formatted according to the given format string. For example if you want to concat "[" and "]" before and after the actual dimension value, you need to specify "[%s]" as format string. -### Filtering DimensionSpecs +### Filtered DimensionSpecs -These are only valid for multi-valued dimensions. If you have a row in druid that has a multi-valued dimension with values ["v1", "v2", "v3"] and you send a groupBy/topN query grouping by that dimension with [query filter](filter.html) for value "v1". In the response you will get 3 rows containing "v1", "v2" and "v3". This behavior might be unintuitive for some use cases. +These are only valid for multi-value dimensions. If you have a row in druid that has a multi-value dimension with values ["v1", "v2", "v3"] and you send a groupBy/topN query grouping by that dimension with [query filter](filter.html) for value "v1". In the response you will get 3 rows containing "v1", "v2" and "v3". This behavior might be unintuitive for some use cases. -It happens because `query filter` is internally used on the bitmaps and only used to match the row to be included in the query result processing. With multivalued dimensions, "query filter" behaves like a contains check, which will match the row with dimension value ["v1", "v2", "v3"]. Please see the section on "Multi-value columns" in [segment](../design/segments.html) for more details. -Then groupBy/topN processing pipeline "explodes" all multi-valued dimensions resulting 3 rows for "v1", "v2" and "v3" each. +It happens because "query filter" is internally used on the bitmaps and only used to match the row to be included in the query result processing. With multi-value dimensions, "query filter" behaves like a contains check, which will match the row with dimension value ["v1", "v2", "v3"]. Please see the section on "Multi-value columns" in [segment](../design/segments.html) for more details. +Then groupBy/topN processing pipeline "explodes" all multi-value dimensions resulting 3 rows for "v1", "v2" and "v3" each. -In addition to "query filter" which efficiently selects the rows to be processed, you can use the filtering dimension spec to filter for specific values within the values of a multi-valued dimension. These dimensionSpecs take a delegate DimensionSpec and a filtering criteria. From the "exploded" rows, only rows matching the given filtering criteria are returned in the query result. +In addition to "query filter" which efficiently selects the rows to be processed, you can use the filtered dimension spec to filter for specific values within the values of a multi-value dimension. These dimensionSpecs take a delegate DimensionSpec and a filtering criteria. From the "exploded" rows, only rows matching the given filtering criteria are returned in the query result. The following filtered dimension spec acts as a whitelist or blacklist for values as per the "isWhitelist" attribute value. @@ -372,7 +372,7 @@ Following filtered dimension spec retains only the values matching regex. Note t { "type" : "regexFiltered", "delegate" : , "pattern": } ``` -For more details and examples, see [multi-valued dimensions](multi-valued-dimensions.html). +For more details and examples, see [multi-value dimensions](multi-value-dimensions.html). ### Upper and Lower extraction functions. diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 910a409abf91..9d6636ed8af6 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -95,3 +95,14 @@ To pull it all together, the above query would return *n\*m* data points, up to ... ] ``` + +### Behavior on multi-value dimensions + +groupBy queries can group on multi-value dimensions. When grouping on a multi-value dimension, _all_ values +from matching rows will be used to generate one group per value. It's possible for a query to return more groups than +there are rows. For example, a groupBy on the dimension `tags` with filter `"t1" OR "t3"` would match only row1, and +generate a result with three groups: `t1`, `t2`, and `t3`. If you only need to include values that match +your filter, you can use a [filtered dimensionSpec](dimensionspecs.html#filtered-dimensionspecs). This can also +improve performance. + +See [Multi-value dimensions](multi-value-dimensions.html) for more details. diff --git a/docs/content/querying/optimizations.md b/docs/content/querying/multi-value-dimensions.md similarity index 59% rename from docs/content/querying/optimizations.md rename to docs/content/querying/multi-value-dimensions.md index d5dc045287b1..adf5bb3a5033 100644 --- a/docs/content/querying/optimizations.md +++ b/docs/content/querying/multi-value-dimensions.md @@ -1,22 +1,38 @@ --- layout: doc_page --- +# Multi-value dimensions -This document contains additional query optimizations for certain types of queries. +Druid supports "multi-value" string dimensions. These are generated when an input field contains an array of values +instead of a single value (e.e. JSON arrays, or a TSV field containing one or more `listDelimiter` characters). -# Multi-value Dimensions +This document describes the behavior of groupBy (topN has similar behavior) queries on multi-value dimensions when they +are used as a dimension being grouped by. See the section on multi-value columns in +[segments](../design/segments.html#multi-value-columns) for internal representation details. -Druid supports "multi-valued" dimensions. See the section on multi-valued columns in [segments](../design/segments.html) for internal representation details. This document describes the behavior of groupBy(topN has similar behavior) queries on multi-valued dimensions when they are used as a dimension being grouped by. +## Querying multi-value dimensions -Suppose, you have a dataSource with a segment that contains following rows with a multi-valued dimension called tags. +Suppose, you have a dataSource with a segment that contains the following rows, with a multi-value dimension +called `tags`. ``` -2772011-01-12T00:00:00.000Z,["t1","t2","t3"], #row1 -2782011-01-13T00:00:00.000Z,["t3","t4","t5"], #row2 -2792011-01-14T00:00:00.000Z,["t5","t6","t7"] #row3 +{"timestamp": "2011-01-12T00:00:00.000Z", "tags": ["t1","t2","t3"]} #row1 +{"timestamp": "2011-01-13T00:00:00.000Z", "tags": ["t3","t4","t5"]} #row2 +{"timestamp": "2011-01-14T00:00:00.000Z", "tags": ["t5","t6","t7"]} #row3 ``` -### Group-By query with no filtering +All query types can filter on multi-value dimensions. Filters operate independently on each value of a multi-value +dimension. For example, a `"t1" OR "t3"` filter would match row1 and row2 but not row3. A `"t1" AND "t3"` filter +would only match row1. + +topN and groupBy queries can group on multi-value dimensions. When grouping on a multi-value dimension, _all_ values +from matching rows will be used to generate one group per value. It's possible for a query to return more groups than +there are rows. For example, a topN on the dimension `tags` with filter `"t1" OR "t3"` would match only row1, and +generate a result with three groups: `t1`, `t2`, and `t3`. If you only need to include values that match +your filter, you can use a [filtered dimensionSpec](dimensionspecs.html#filtered-dimensionspecs). This can also +improve performance. + +### Example: GroupBy query with no filtering See [GroupBy querying](groupbyquery.html) for details. @@ -104,7 +120,7 @@ returns following result. notice how original rows are "exploded" into multiple rows and merged. -### Group-By query with a selector query filter +### Example: GroupBy query with a selector query filter See [query filters](filters.html) for details of selector query filter. @@ -181,13 +197,13 @@ returns following result. ] ``` -You might be surprised to see inclusion of "t1", "t2", "t4" and "t5" in the results. It happens because query filter is applied on the row before explosion. For multi-valued dimensions, selector filter for "t3" would match row1 and row2, after which exploding is done. For multi-valued dimensions, query filter matches a row if any individual value inside the multiple values matches the query filter. +You might be surprised to see inclusion of "t1", "t2", "t4" and "t5" in the results. It happens because query filter is applied on the row before explosion. For multi-value dimensions, selector filter for "t3" would match row1 and row2, after which exploding is done. For multi-value dimensions, query filter matches a row if any individual value inside the multiple values matches the query filter. -### Group-By query with a selector query filter and additional filter in "dimensions" attributes +### Example: GroupBy query with a selector query filter and additional filter in "dimensions" attributes To solve the problem above and to get only rows for "t3" returned, you would have to use a "filtered dimension spec" as in the query below. -See section on filtered dimensionSpecs in [dimensionSpecs](dimensionspecs.html) for details. +See section on filtered dimensionSpecs in [dimensionSpecs](dimensionspecs.html#filtered-dimensionspecs) for details. ```json { @@ -224,7 +240,7 @@ See section on filtered dimensionSpecs in [dimensionSpecs](dimensionspecs.html) } ``` -returns following result. +returns the following result. ```json [ @@ -238,5 +254,4 @@ returns following result. ] ``` -Note that, for groupBy queries, you could get similar result with a [having spec](having.html) but using a filtered dimensionSpec would be much more efficient because that gets applied at the lowest level in the query processing pipeline while having spec is applied at the highest level of groupBy query processing. - +Note that, for groupBy queries, you could get similar result with a [having spec](having.html) but using a filtered dimensionSpec is much more efficient because that gets applied at the lowest level in the query processing pipeline. Having specs are applied at the outermost level of groupBy query processing. diff --git a/docs/content/querying/topnquery.md b/docs/content/querying/topnquery.md index 7de0d49f6e4a..f9aa00ca8569 100644 --- a/docs/content/querying/topnquery.md +++ b/docs/content/querying/topnquery.md @@ -128,7 +128,20 @@ The format of the results would look like so: } ] ``` + +### Behavior on multi-value dimensions + +topN queries can group on multi-value dimensions. When grouping on a multi-value dimension, _all_ values +from matching rows will be used to generate one group per value. It's possible for a query to return more groups than +there are rows. For example, a topN on the dimension `tags` with filter `"t1" OR "t3"` would match only row1, and +generate a result with three groups: `t1`, `t2`, and `t3`. If you only need to include values that match +your filter, you can use a [filtered dimensionSpec](dimensionspecs.html#filtered-dimensionspecs). This can also +improve performance. + +See [Multi-value dimensions](multi-value-dimensions.html) for more details. + ### Aliasing + The current TopN algorithm is an approximate algorithm. The top 1000 local results from each segment are returned for merging to determine the global topN. As such, the topN algorithm is approximate in both rank and results. Approximate results *ONLY APPLY WHEN THERE ARE MORE THAN 1000 DIM VALUES*. A topN over a dimension with fewer than 1000 unique dimension values can be considered accurate in rank and accurate in aggregates. The threshold can be modified from it's default 1000 via the server parameter `druid.query.topN.minTopNThreshold` which need to restart servers to take effect or set `minTopNThreshold` in query context which take effect per query. diff --git a/docs/content/toc.md b/docs/content/toc.md index 81b7061ceca2..bb0582c0e9c0 100644 --- a/docs/content/toc.md +++ b/docs/content/toc.md @@ -40,9 +40,9 @@ * [Granularities](../querying/granularities.html) * [DimensionSpecs](../querying/dimensionspecs.html) * [Context](../querying/query-context.html) + * [Multi-value dimensions](../querying/multi-value-dimensions.html) * [SQL](../querying/sql.html) * [Joins](../querying/joins.html) - * [Optimizations](../querying/optimizations.html) * [Multitenancy](../querying/multitenancy.html) * [Caching](../querying/caching.html) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java index 978919548774..c57c58bba6ad 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java @@ -601,7 +601,7 @@ protected void innerReduce( // Respect poisoning if (!currentDimSkip && dvc.numRows < 0) { - log.info("Cannot partition on multi-valued dimension: %s", dvc.dim); + log.info("Cannot partition on multi-value dimension: %s", dvc.dim); currentDimSkip = true; } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 5417d7c4ea94..8c34ad7a2ece 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -591,7 +591,7 @@ public ObjectColumnSelector makeObjectColumnSelector(String column) if (columnVals.hasMultipleValues()) { throw new UnsupportedOperationException( - "makeObjectColumnSelector does not support multivalued GenericColumns" + "makeObjectColumnSelector does not support multi-value GenericColumns" ); } From e3435009571ecad6f55800929720fe606ea70064 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 22 Mar 2016 13:54:49 -0700 Subject: [PATCH 2/3] refactor extensions into their own docs (Backport of #2698) --- docs/content/dependencies/deep-storage.md | 22 +- docs/content/dependencies/metadata-storage.md | 129 +-------- .../azure.md | 0 .../cassandra.md | 0 .../cloudfiles.md | 0 .../graphite.md | 0 .../kafka-simple.md | 0 .../rabbitmq.md | 0 .../approximate-histograms.md | 4 +- .../development/extensions-core/avro.md | 114 ++++++++ .../datasketches-aggregators.md | 32 +-- .../development/extensions-core/examples.md | 25 ++ .../development/extensions-core/hdfs.md | 18 ++ .../extensions-core/kafka-eight-firehose.md | 34 +++ .../kafka-extraction-namespace.md | 60 ++++ .../development/extensions-core/mysql.md | 55 ++++ .../extensions-core/namespaced-lookup.md | 239 +++++++++++++++ .../development/extensions-core/postgresql.md | 49 ++++ .../content/development/extensions-core/s3.md | 38 +++ docs/content/development/extensions.md | 32 +-- docs/content/ingestion/data-formats.md | 2 + docs/content/ingestion/firehose.md | 76 +---- docs/content/ingestion/index.md | 109 +------ docs/content/querying/lookups.md | 272 +----------------- docs/content/toc.md | 4 +- extensions-contrib/README.md | 2 +- 26 files changed, 700 insertions(+), 616 deletions(-) rename docs/content/development/{community-extensions => extensions-contrib}/azure.md (100%) rename docs/content/development/{community-extensions => extensions-contrib}/cassandra.md (100%) rename docs/content/development/{community-extensions => extensions-contrib}/cloudfiles.md (100%) rename docs/content/development/{community-extensions => extensions-contrib}/graphite.md (100%) rename docs/content/development/{community-extensions => extensions-contrib}/kafka-simple.md (100%) rename docs/content/development/{community-extensions => extensions-contrib}/rabbitmq.md (100%) rename docs/content/development/{ => extensions-core}/approximate-histograms.md (97%) create mode 100644 docs/content/development/extensions-core/avro.md rename docs/content/development/{ => extensions-core}/datasketches-aggregators.md (75%) create mode 100644 docs/content/development/extensions-core/examples.md create mode 100644 docs/content/development/extensions-core/hdfs.md create mode 100644 docs/content/development/extensions-core/kafka-eight-firehose.md create mode 100644 docs/content/development/extensions-core/kafka-extraction-namespace.md create mode 100644 docs/content/development/extensions-core/mysql.md create mode 100644 docs/content/development/extensions-core/namespaced-lookup.md create mode 100644 docs/content/development/extensions-core/postgresql.md create mode 100644 docs/content/development/extensions-core/s3.md diff --git a/docs/content/dependencies/deep-storage.md b/docs/content/dependencies/deep-storage.md index 880242e74c23..ca386398ac84 100644 --- a/docs/content/dependencies/deep-storage.md +++ b/docs/content/dependencies/deep-storage.md @@ -1,7 +1,9 @@ --- layout: doc_page --- + # Deep Storage + Deep storage is where segments are stored. It is a storage mechanism that Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid nodes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented. ## Local Mount @@ -21,24 +23,12 @@ If you are using the Hadoop indexer in local mode, then just give it a local fil ## S3-compatible -S3-compatible deep storage is basically either S3 or something like Google Storage which exposes the same API as S3. - -S3 configuration parameters are - -|Property|Possible Values|Description|Default| -|--------|---------------|-----------|-------| -|`druid.s3.accessKey`||S3 access key.|Must be set.| -|`druid.s3.secretKey`||S3 secret key.|Must be set.| -|`druid.storage.bucket`||Bucket to store in.|Must be set.| -|`druid.storage.baseKey`||Base key prefix to use, i.e. what directory.|Must be set.| +See [druid-s3-extensions extension documentation](../development/extensions-core/s3.html). ## HDFS -In order to use hdfs for deep storage, you need to set the following configuration in your common configs. +See [druid-hdfs-storage extension documentation](../development/extensions-core/hdfs.html). -|Property|Possible Values|Description|Default| -|--------|---------------|-----------|-------| -|`druid.storage.type`|hdfs||Must be set.| -|`druid.storage.storageDirectory`||Directory for storing segments.|Must be set.| +## Additional Deep Stores -If you are using the Hadoop indexer, set your output directory to be a location on Hadoop and it will work +For additional deep stores, please see our [extensions list](../development/extensions.html). diff --git a/docs/content/dependencies/metadata-storage.md b/docs/content/dependencies/metadata-storage.md index 16add3b7cdbc..ddf8855e7b7c 100644 --- a/docs/content/dependencies/metadata-storage.md +++ b/docs/content/dependencies/metadata-storage.md @@ -1,128 +1,19 @@ --- layout: doc_page --- + # Metadata Storage The Metadata Storage is an external dependency of Druid. Druid uses it to store various metadata about the system, but not to store the actual data. There are a number of tables used for various purposes described below. -## Supported Metadata Storages - -The following metadata storage engines are supported: - -* Derby (default, but not suitable for production) -* MySQL -* PostgreSQL - -Even though Derby is the default, it works only if you have all Druid -processes running on the same host, and should be used only for experimentation. -For production, MySQL or PostgreSQL should be used. - -To choose the metadata storage type, set `druid.metadata.storage.type` to -`mysql`, `postgres` or `derby`. -Set other `druid.metadata.storage` configuration -keywords as shown below to give Druid information about how to connect to -the database. - -As discussed in [Including Extensions](../operations/including-extensions.html), -there are two ways for giving Druid the extension files it needs for the -database you are using. -The first is to put the extension files in the classpath. The second is to -put the extension files in a subdirectory of -`druid.extensions.directory` (by default `extensions` under the Druid working directory) and list the subdirectory name in -`druid.extensions.loadList`. The example properties below show the second -way. - -## Setting up MySQL - -1. Install MySQL - - Use your favorite package manager to install mysql, e.g.: - - on Ubuntu/Debian using apt `apt-get install mysql-server` - - on OS X, using [Homebrew](http://brew.sh/) `brew install mysql` - - Alternatively, download and follow installation instructions for MySQL - Community Server here: - [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/) - -2. Create a druid database and user - - Connect to MySQL from the machine where it is installed. - - ```bash - > mysql -u root - ``` - - Paste the following snippet into the mysql prompt: - - ```sql - -- create a druid database, make sure to use utf8 as encoding - CREATE DATABASE druid DEFAULT CHARACTER SET utf8; - - -- create a druid user, and grant it all permission on the database we just created - GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd'; - ``` - -3. Configure your Druid metadata storage extension: - - Add the following parameters to your Druid configuration, replacing `` - with the location (host name and port) of the database. +Derby is the default metadata store for Druid, however, it is not suitable for production. +[MySQL](../development/extensions-core/mysql.html) and [PostgreSQL](../development/extensions-core/postgresql.html) are more production suitable metadata stores. - ```properties - druid.extensions.loadList=["mysql-metadata-storage"] - druid.metadata.storage.type=mysql - druid.metadata.storage.connector.connectURI=jdbc:mysql:///druid - druid.metadata.storage.connector.user=druid - druid.metadata.storage.connector.password=diurd - ``` - - Note: the metadata storage extension is not packaged within the main Druid tarball; it is - packaged in a separate tarball that can be downloaded from [here](http://druid.io/downloads.html). - You can also get it using [pull-deps](../pull-deps.html), or you can build - it from source code; see [Build from Source](../development/build.html). - -## Setting up PostgreSQL - -1. Install PostgreSQL - - Use your favorite package manager to install PostgreSQL, e.g.: - - on Ubuntu/Debian using apt `apt-get install postgresql` - - on OS X, using [Homebrew](http://brew.sh/) `brew install postgresql` - -2. Create a druid database and user - - On the machine where PostgreSQL is installed, using an account with proper - postgresql permissions: - - Create a druid user, enter `diurd` when prompted for the password. - - ```bash - createuser druid -P - ``` - - Create a druid database owned by the user we just created - - ```bash - createdb druid -O druid - ``` - - *Note:* On Ubuntu / Debian you may have to prefix the `createuser` and - `createdb` commands with `sudo -u postgres` in order to gain proper - permissions. - -3. Configure your Druid metadata storage extension: - - Add the following parameters to your Druid configuration, replacing `` - with the location (host name and port) of the database. - - ```properties - druid.extensions.loadList=["postgresql-metadata-storage"] - druid.metadata.storage.type=postgresql - druid.metadata.storage.connector.connectURI=jdbc:postgresql:///druid - druid.metadata.storage.connector.user=druid - druid.metadata.storage.connector.password=diurd - ``` +
+Derby is not suitable for production use as a metadata store. Use MySQL or PostgreSQL instead. +
## Using derby @@ -132,6 +23,14 @@ way. druid.metadata.storage.type=derby druid.metadata.storage.connector.connectURI=jdbc:derby://localhost:1527//home/y/var/druid_state/derby;create=true ``` + +## MySQL + +See [mysql-metadata-storage extension documentation](../development/extensions-core/mysql.html). + +## PostgreSQL + +See [postgresql-metadata-storage](../development/extensions-core/postgresql.html). ## Metadata Storage Tables diff --git a/docs/content/development/community-extensions/azure.md b/docs/content/development/extensions-contrib/azure.md similarity index 100% rename from docs/content/development/community-extensions/azure.md rename to docs/content/development/extensions-contrib/azure.md diff --git a/docs/content/development/community-extensions/cassandra.md b/docs/content/development/extensions-contrib/cassandra.md similarity index 100% rename from docs/content/development/community-extensions/cassandra.md rename to docs/content/development/extensions-contrib/cassandra.md diff --git a/docs/content/development/community-extensions/cloudfiles.md b/docs/content/development/extensions-contrib/cloudfiles.md similarity index 100% rename from docs/content/development/community-extensions/cloudfiles.md rename to docs/content/development/extensions-contrib/cloudfiles.md diff --git a/docs/content/development/community-extensions/graphite.md b/docs/content/development/extensions-contrib/graphite.md similarity index 100% rename from docs/content/development/community-extensions/graphite.md rename to docs/content/development/extensions-contrib/graphite.md diff --git a/docs/content/development/community-extensions/kafka-simple.md b/docs/content/development/extensions-contrib/kafka-simple.md similarity index 100% rename from docs/content/development/community-extensions/kafka-simple.md rename to docs/content/development/extensions-contrib/kafka-simple.md diff --git a/docs/content/development/community-extensions/rabbitmq.md b/docs/content/development/extensions-contrib/rabbitmq.md similarity index 100% rename from docs/content/development/community-extensions/rabbitmq.md rename to docs/content/development/extensions-contrib/rabbitmq.md diff --git a/docs/content/development/approximate-histograms.md b/docs/content/development/extensions-core/approximate-histograms.md similarity index 97% rename from docs/content/development/approximate-histograms.md rename to docs/content/development/extensions-core/approximate-histograms.md index 276109ee3139..7392efd4141a 100644 --- a/docs/content/development/approximate-histograms.md +++ b/docs/content/development/extensions-core/approximate-histograms.md @@ -2,7 +2,9 @@ layout: doc_page --- -### Approximate Histogram aggregator +# Approximate Histogram aggregator + +Make sure to [include](../../operations/including-extensions.html) `druid-histogram` as an extension. This aggregator is based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf) diff --git a/docs/content/development/extensions-core/avro.md b/docs/content/development/extensions-core/avro.md new file mode 100644 index 000000000000..204b36f2a13c --- /dev/null +++ b/docs/content/development/extensions-core/avro.md @@ -0,0 +1,114 @@ +--- +layout: doc_page +--- + +# Avro + +This extension enables Druid to ingest and understand the Apache Avro data format. Make sure to [include](../../operations/including-extensions.html) `druid-avro-extensions` as an extension. + +### Avro Stream Parser + +This is for streaming/realtime ingestion. + +| Field | Type | Description | Required | +|-------|------|-------------|----------| +| type | String | This should say `avro_stream`. | no | +| avroBytesDecoder | JSON Object | Specifies how to decode bytes to Avro record. | yes | +| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes | + +For example, using Avro stream parser with schema repo Avro bytes decoder: + +```json +"parser" : { + "type" : "avro_stream", + "avroBytesDecoder" : { + "type" : "schema_repo", + "subjectAndIdConverter" : { + "type" : "avro_1124", + "topic" : "${YOUR_TOPIC}" + }, + "schemaRepository" : { + "type" : "avro_1124_rest_client", + "url" : "${YOUR_SCHEMA_REPO_END_POINT}", + } + }, + "parseSpec" : { + "type": "timeAndDims", + "timestampSpec": , + "dimensionsSpec": + } +} +``` + +#### Avro Bytes Decoder + +If `type` is not included, the avroBytesDecoder defaults to `schema_repo`. + +##### SchemaRepo Based Avro Bytes Decoder + +This Avro bytes decoder first extract `subject` and `id` from input message bytes, then use them to lookup the Avro schema with which to decode Avro record from bytes. Details can be found in [schema repo](https://github.com/schema-repo/schema-repo) and [AVRO-1124](https://issues.apache.org/jira/browse/AVRO-1124). You will need an http service like schema repo to hold the avro schema. Towards schema registration on the message producer side, you can refer to `io.druid.data.input.AvroStreamInputRowParserTest#testParse()`. + +| Field | Type | Description | Required | +|-------|------|-------------|----------| +| type | String | This should say `schema_repo`. | no | +| subjectAndIdConverter | JSON Object | Specifies the how to extract subject and id from message bytes. | yes | +| schemaRepository | JSON Object | Specifies the how to lookup Avro schema from subject and id. | yes | + +##### Avro-1124 Subject And Id Converter + +| Field | Type | Description | Required | +|-------|------|-------------|----------| +| type | String | This should say `avro_1124`. | no | +| topic | String | Specifies the topic of your kafka stream. | yes | + + +##### Avro-1124 Schema Repository + +| Field | Type | Description | Required | +|-------|------|-------------|----------| +| type | String | This should say `avro_1124_rest_client`. | no | +| url | String | Specifies the endpoint url of your Avro-1124 schema repository. | yes | + +### Avro Hadoop Parser + +This is for batch ingestion using the HadoopDruidIndexer. The `inputFormat` of `inputSpec` in `ioConfig` must be set to `"io.druid.data.input.avro.AvroValueInputFormat"`. You may want to set Avro reader's schema in `jobProperties` in `tuningConfig`, eg: `"avro.schema.path.input.value": "/path/to/your/schema.avsc"` or `"avro.schema.input.value": "your_schema_JSON_object"`, if reader's schema is not set, the schema in Avro object container file will be used, see [Avro specification](http://avro.apache.org/docs/1.7.7/spec.html#Schema+Resolution). Make sure to include "io.druid.extensions:druid-avro-extensions" as an extension. + +| Field | Type | Description | Required | +|-------|------|-------------|----------| +| type | String | This should say `avro_hadoop`. | no | +| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes | +| fromPigAvroStorage | Boolean | Specifies whether the data file is stored using AvroStorage. | no(default == false) | + +For example, using Avro Hadoop parser with custom reader's schema file: + +```json +{ + "type" : "index_hadoop", + "spec" : { + "dataSchema" : { + "dataSource" : "", + "parser" : { + "type" : "avro_hadoop", + "parseSpec" : { + "type": "timeAndDims", + "timestampSpec": , + "dimensionsSpec": + } + } + }, + "ioConfig" : { + "type" : "hadoop", + "inputSpec" : { + "type" : "static", + "inputFormat": "io.druid.data.input.avro.AvroValueInputFormat", + "paths" : "" + } + }, + "tuningConfig" : { + "jobProperties" : { + "avro.schema.path.input.value" : "/path/to/my/schema.avsc", + } + } + } +} +``` diff --git a/docs/content/development/datasketches-aggregators.md b/docs/content/development/extensions-core/datasketches-aggregators.md similarity index 75% rename from docs/content/development/datasketches-aggregators.md rename to docs/content/development/extensions-core/datasketches-aggregators.md index f093678a88af..147bafc30ce2 100644 --- a/docs/content/development/datasketches-aggregators.md +++ b/docs/content/development/extensions-core/datasketches-aggregators.md @@ -8,32 +8,32 @@ Druid aggregators based on [datasketches](http://datasketches.github.io/) librar At ingestion time, this aggregator creates the theta sketch objects which get stored in Druid segments. Logically speaking, a theta sketch object can be thought of as a Set data structure. At query time, sketches are read and aggregated (set unioned) together. In the end, by default, you receive the estimate of the number of unique entries in the sketch object. Also, you can use post aggregators to do union, intersection or difference on sketch columns in the same row. Note that you can use `thetaSketch` aggregator on columns which were not ingested using same, it will return estimated cardinality of the column. It is recommended to use it at ingestion time as well to make querying faster. +To use the datasketch aggregators, make sure you [include](../../operations/including-extensions.html) the extension in your config file: + +``` +druid.extensions.loadList=["druid-datasketches"] +``` + ### Aggregators ```json { "type" : "thetaSketch", "name" : , - "fieldName" : , - - //following boolean field is optional. This should only be used at - //indexing time if your input data contains theta sketch objects. - //that would be the case if you use datasketches library outside of Druid, - //say with Pig/Hive, to produce the data that you are ingesting into Druid - "isInputThetaSketch": false - - //following field is optional, default = 16384. must be a power of 2. - //Internally, size refers to the maximum number - //of entries sketch object will retain, higher size would mean higher - //accuracy but higher space needed to store those sketches. - //note that after you index with a particular size, druid will persist sketch in segments - //and you will use size greater or equal to that at query time. - //See [theta-size](http://datasketches.github.io/docs/ThetaSize.html) for details. - //In general, We recommend just sticking to default size, which has worked well. + "fieldName" : , + "isInputThetaSketch": false, "size": 16384 } ``` +|property|description|required?| +|--------|-----------|---------| +|type|This String should always be "thetaSketch"|yes| +|name|A String for the output (result) name of the calculation.|yes| +|fieldName|A String for the name of the aggregator used at ingestion time.|yes| +|isInputThetaSketch|This should only be used at indexing time if your input data contains theta sketch objects. This would be the case if you use datasketches library outside of Druid, say with Pig/Hive, to produce the data that you are ingesting into Druid |no, defaults to false| +|size|Must be a power of 2. Internally, size refers to the maximum number of entries sketch object will retain. Higher size means higher accuracy but more space to store sketches. Note that after you index with a particular size, druid will persist sketch in segments and you will use size greater or equal to that at query time. See [theta-size](http://datasketches.github.io/docs/ThetaSize.html) for details. In general, We recommend just sticking to default size. |no, defaults to 16384| + ### Post Aggregators #### Sketch Estimator diff --git a/docs/content/development/extensions-core/examples.md b/docs/content/development/extensions-core/examples.md new file mode 100644 index 000000000000..cd4314cc929b --- /dev/null +++ b/docs/content/development/extensions-core/examples.md @@ -0,0 +1,25 @@ +--- +layout: doc_page +--- + +# Druid examples + +## TwitterSpritzerFirehose + +This firehose connects directly to the twitter spritzer data stream. + +Sample spec: + +```json +"firehose" : { + "type" : "twitzer", + "maxEventCount": -1, + "maxRunMinutes": 0 +} +``` + +|property|description|default|required?| +|--------|-----------|-------|---------| +|type|This should be "twitzer"|N/A|yes| +|maxEventCount|max events to receive, -1 is infinite, 0 means nothing is delivered; use this to prevent infinite space consumption or to prevent getting throttled at an inconvenient time.|N/A|yes| +|maxRunMinutes|maximum number of minutes to fetch Twitter events. Use this to prevent getting throttled at an inconvenient time. If zero or less, no time limit for run.|N/A|yes| diff --git a/docs/content/development/extensions-core/hdfs.md b/docs/content/development/extensions-core/hdfs.md new file mode 100644 index 000000000000..177676554e54 --- /dev/null +++ b/docs/content/development/extensions-core/hdfs.md @@ -0,0 +1,18 @@ +--- +layout: doc_page +--- + +# HDFS + +Make sure to [include](../../operations/including-extensions.html) `druid-hdfs-storage` as an extension. + +## Deep Storage + +### Configuration + +|Property|Possible Values|Description|Default| +|--------|---------------|-----------|-------| +|`druid.storage.type`|hdfs||Must be set.| +|`druid.storage.storageDirectory`||Directory for storing segments.|Must be set.| + +If you are using the Hadoop indexer, set your output directory to be a location on Hadoop and it will work diff --git a/docs/content/development/extensions-core/kafka-eight-firehose.md b/docs/content/development/extensions-core/kafka-eight-firehose.md new file mode 100644 index 000000000000..7e5ecb9ac5e0 --- /dev/null +++ b/docs/content/development/extensions-core/kafka-eight-firehose.md @@ -0,0 +1,34 @@ +--- +layout: doc_page +--- + +# Kafka Eight Firehose + +Make sure to [include](../../operations/including-extensions.html) `druid-kafka-eight` as an extension. + +This firehose acts as a Kafka 0.8.x consumer and ingests data from Kafka. + +Sample spec: + +```json +"firehose": { + "type": "kafka-0.8", + "consumerProps": { + "zookeeper.connect": "localhost:2181", + "zookeeper.connection.timeout.ms" : "15000", + "zookeeper.session.timeout.ms" : "15000", + "zookeeper.sync.time.ms" : "5000", + "group.id": "druid-example", + "fetch.message.max.bytes" : "1048586", + "auto.offset.reset": "largest", + "auto.commit.enable": "false" + }, + "feed": "wikipedia" +} +``` + +|property|description|required?| +|--------|-----------|---------| +|type|This should be "kafka-0.8"|yes| +|consumerProps|The full list of consumer configs can be [here](https://kafka.apache.org/08/configuration.html).|yes| +|feed|Kafka maintains feeds of messages in categories called topics. This is the topic name.|yes| diff --git a/docs/content/development/extensions-core/kafka-extraction-namespace.md b/docs/content/development/extensions-core/kafka-extraction-namespace.md new file mode 100644 index 000000000000..0b82fa9612ba --- /dev/null +++ b/docs/content/development/extensions-core/kafka-extraction-namespace.md @@ -0,0 +1,60 @@ +--- +layout: doc_page +--- + +# Kafka Namespaced Lookup + +
+Lookups are an experimental feature. +
+ +Make sure to [include](../../operations/including-extensions.html) `druid-namespace-lookup` and `druid-kafka-extraction-namespace` as an extension. + +Note that this lookup does not employ a `pollPeriod`. + +If you need updates to populate as promptly as possible, it is possible to plug into a kafka topic whose key is the old value and message is the desired new value (both in UTF-8).\ + +```json +{ + "type":"kafka", + "namespace":"testTopic", + "kafkaTopic":"testTopic" +} +``` + +|Parameter|Description|Required|Default| +|---------|-----------|--------|-------| +|`namespace`|The namespace to define|Yes|| +|`kafkaTopic`|The kafka topic to read the data from|Yes|| + +## Kafka renames + +The extension `kafka-extraction-namespace` enables reading from a kafka feed which has name/key pairs to allow renaming of dimension values. An example use case would be to rename an ID to a human readable format. + +Currently the historical node caches the key/value pairs from the kafka feed in an ephemeral memory mapped DB via MapDB. + +## Configuration + +The following options are used to define the behavior and should be included wherever the extension is included (all query servicing nodes): + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.rename.kafka.properties`|A json map of kafka consumer properties. See below for special properties.|See below| + +The following are the handling for kafka consumer properties in `druid.query.rename.kafka.properties` + +|Property|Description|Default| +|--------|-----------|-------| +|`zookeeper.connect`|Zookeeper connection string|`localhost:2181/kafka`| +|`group.id`|Group ID, auto-assigned for publish-subscribe model and cannot be overridden|`UUID.randomUUID().toString()`| +|`auto.offset.reset`|Setting to get the entire kafka rename stream. Cannot be overridden|`smallest`| + +## Testing the Kafka rename functionality + +To test this setup, you can send key/value pairs to a kafka stream via the following producer console: + +``` +./bin/kafka-console-producer.sh --property parse.key=true --property key.separator="->" --broker-list localhost:9092 --topic testTopic +``` + +Renames can then be published as `OLD_VAL->NEW_VAL` followed by newline (enter or return) diff --git a/docs/content/development/extensions-core/mysql.md b/docs/content/development/extensions-core/mysql.md new file mode 100644 index 000000000000..a63291d7c5d8 --- /dev/null +++ b/docs/content/development/extensions-core/mysql.md @@ -0,0 +1,55 @@ +--- +layout: doc_page +--- + +# MySQL Metadata Store + +Make sure to [include](../../operations/including-extensions.html) `mysql-metadata-storage` as an extension. + +## Setting up MySQL + +1. Install MySQL + + Use your favorite package manager to install mysql, e.g.: + - on Ubuntu/Debian using apt `apt-get install mysql-server` + - on OS X, using [Homebrew](http://brew.sh/) `brew install mysql` + + Alternatively, download and follow installation instructions for MySQL + Community Server here: + [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/) + +2. Create a druid database and user + + Connect to MySQL from the machine where it is installed. + + ```bash + > mysql -u root + ``` + + Paste the following snippet into the mysql prompt: + + ```sql + -- create a druid database, make sure to use utf8 as encoding + CREATE DATABASE druid DEFAULT CHARACTER SET utf8; + + -- create a druid user, and grant it all permission on the database we just created + GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd'; + ``` + +3. Configure your Druid metadata storage extension: + + Add the following parameters to your Druid configuration, replacing `` + with the location (host name and port) of the database. + + ```properties + druid.extensions.loadList=["mysql-metadata-storage"] + druid.metadata.storage.type=mysql + druid.metadata.storage.connector.connectURI=jdbc:mysql:///druid + druid.metadata.storage.connector.user=druid + druid.metadata.storage.connector.password=diurd + ``` + + Note: the metadata storage extension is not packaged within the main Druid tarball; it is + packaged in a separate tarball that can be downloaded from [here](http://druid.io/downloads.html). + You can also get it using [pull-deps](../pull-deps.html), or you can build + it from source code; see [Build from Source](../development/build.html). diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md new file mode 100644 index 000000000000..8479aa5c8348 --- /dev/null +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -0,0 +1,239 @@ +--- +layout: doc_page +--- + +# Namespaced Lookup + +
+Lookups are an experimental feature. +
+ +Make sure to [include](../../operations/including-extensions.html) `druid-namespace-lookup` as an extension. + +## Configuration + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.lookup.snapshotWorkingDir`| Working path used to store snapshot of current lookup configuration, leaving this property null will disable snapshot/bootstrap utility|null| + +Namespaced lookups are appropriate for lookups which are not possible to pass at query time due to their size, +or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers. +Namespaced lookups can be specified as part of the runtime properties file. The property is a list of the namespaces +described as per the sections on this page. For example: + + ```json + druid.query.extraction.namespace.lookups= + [ + { + "type": "uri", + "namespace": "some_uri_lookup", + "uri": "file:/tmp/prefix/", + "namespaceParseSpec": { + "format": "csv", + "columns": [ + "key", + "value" + ] + }, + "pollPeriod": "PT5M" + }, + { + "type": "jdbc", + "namespace": "some_jdbc_lookup", + "connectorConfig": { + "createTables": true, + "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", + "user": "druid", + "password": "diurd" + }, + "table": "lookupTable", + "keyColumn": "mykeyColumn", + "valueColumn": "MyValueColumn", + "tsColumn": "timeColumn" + } + ] + ``` + +Proper functionality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes: +`druid-namespace-lookup` + +## Cache Settings + +Lookups are cached locally on historical nodes. The following are settings used by the nodes which service queries when +setting namespaces (broker, peon, historical) + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.extraction.namespace.cache.type`|Specifies the type of caching to be used by the namespaces. May be one of [`offHeap`, `onHeap`]. `offHeap` uses a temporary file for off-heap storage of the namespace (memory mapped files). `onHeap` stores all cache on the heap in standard java map types.|`onHeap`| + +The cache is populated in different ways depending on the settings below. In general, most namespaces employ +a `pollPeriod` at the end of which time they poll the remote resource of interest for updates. + +# Supported Lookups + +For additional lookups, please see our [extensions list](../development/extensions.html). + +## URI namespace update + +The remapping values for each namespaced lookup can be specified by json as per + +```json +{ + "type":"uri", + "namespace":"some_lookup", + "uri": "s3://bucket/some/key/prefix/", + "namespaceParseSpec":{ + "format":"csv", + "columns":["key","value"] + }, + "pollPeriod":"PT5M", + "versionRegex": "renames-[0-9]*\\.gz" +} +``` + +|Property|Description|Required|Default| +|--------|-----------|--------|-------| +|`namespace`|The namespace to define|Yes|| +|`pollPeriod`|Period between polling for updates|No|0 (only once)| +|`versionRegex`|Regex to help find newer versions of the namespace data|Yes|| +|`namespaceParseSpec`|How to interpret the data at the URI|Yes|| + +The `pollPeriod` value specifies the period in ISO 8601 format between checks for updates. If the source of the lookup is capable of providing a timestamp, the lookup will only be updated if it has changed since the prior tick of `pollPeriod`. A value of 0, an absent parameter, or `null` all mean populate once and do not attempt to update. Whenever an update occurs, the updating system will look for a file with the most recent timestamp and assume that one with the most recent data. + +The `versionRegex` value specifies a regex to use to determine if a filename in the parent path of the uri should be considered when trying to find the latest version. Omitting this setting or setting it equal to `null` will match to all files it can find (equivalent to using `".*"`). The search occurs in the most significant "directory" of the uri. + +The `namespaceParseSpec` can be one of a number of values. Each of the examples below would rename foo to bar, baz to bat, and buck to truck. All parseSpec types assumes each input is delimited by a new line. See below for the types of parseSpec supported. + +### csv lookupParseSpec + +|Parameter|Description|Required|Default| +|---------|-----------|--------|-------| +|`columns`|The list of columns in the csv file|yes|`null`| +|`keyColumn`|The name of the column containing the key|no|The first column| +|`valueColumn`|The name of the column containing the value|no|The second column| + +*example input* + +``` +bar,something,foo +bat,something2,baz +truck,something3,buck +``` + +*example namespaceParseSpec* + +```json +"namespaceParseSpec": { + "format": "csv", + "columns": ["value","somethingElse","key"], + "keyColumn": "key", + "valueColumn": "value" +} +``` + +### tsv lookupParseSpec + +|Parameter|Description|Required|Default| +|---------|-----------|--------|-------| +|`columns`|The list of columns in the csv file|yes|`null`| +|`keyColumn`|The name of the column containing the key|no|The first column| +|`valueColumn`|The name of the column containing the value|no|The second column| +|`delimiter`|The delimiter in the file|no|tab (`\t`)| + + +*example input* + +``` +bar|something,1|foo +bat|something,2|baz +truck|something,3|buck +``` + +*example namespaceParseSpec* + +```json +"namespaceParseSpec": { + "format": "tsv", + "columns": ["value","somethingElse","key"], + "keyColumn": "key", + "valueColumn": "value", + "delimiter": "|" +} +``` + +### customJson lookupParseSpec + +|Parameter|Description|Required|Default| +|---------|-----------|--------|-------| +|`keyFieldName`|The field name of the key|yes|null| +|`valueFieldName`|The field name of the value|yes|null| + +*example input* + +```json +{"key": "foo", "value": "bar", "somethingElse" : "something"} +{"key": "baz", "value": "bat", "somethingElse" : "something"} +{"key": "buck", "somethingElse": "something", "value": "truck"} +``` + +*example namespaceParseSpec* + +```json +"namespaceParseSpec": { + "format": "customJson", + "keyFieldName": "key", + "valueFieldName": "value" +} +``` + + +### simpleJson lookupParseSpec +The `simpleJson` lookupParseSpec does not take any parameters. It is simply a line delimited json file where the field is the key, and the field's value is the value. + +*example input* + +```json +{"foo": "bar"} +{"baz": "bat"} +{"buck": "truck"} +``` + +*example namespaceParseSpec* + +```json +"namespaceParseSpec":{ + "format": "simpleJson" +} +``` + +## JDBC namespaced lookup + +The JDBC lookups will poll a database to populate its local cache. If the `tsColumn` is set it must be able to accept comparisons in the format `'2015-01-01 00:00:00'`. For example, the following must be valid sql for the table `SELECT * FROM some_lookup_table WHERE timestamp_column > '2015-01-01 00:00:00'`. If `tsColumn` is set, the caching service will attempt to only poll values that were written *after* the last sync. If `tsColumn` is not set, the entire table is pulled every time. + +|Parameter|Description|Required|Default| +|---------|-----------|--------|-------| +|`namespace`|The namespace to define|Yes|| +|`connectorConfig`|The connector config to use|Yes|| +|`table`|The table which contains the key value pairs|Yes|| +|`keyColumn`|The column in `table` which contains the keys|Yes|| +|`valueColumn`|The column in `table` which contains the values|Yes|| +|`tsColumn`| The column in `table` which contains when the key was updated|No|Not used| +|`pollPeriod`|How often to poll the DB|No|0 (only once)| + +```json +{ + "type":"jdbc", + "namespace":"some_lookup", + "connectorConfig":{ + "createTables":true, + "connectURI":"jdbc:mysql://localhost:3306/druid", + "user":"druid", + "password":"diurd" + }, + "table":"some_lookup_table", + "keyColumn":"the_old_dim_value", + "valueColumn":"the_new_dim_value", + "tsColumn":"timestamp_column", + "pollPeriod":600000 +} +``` diff --git a/docs/content/development/extensions-core/postgresql.md b/docs/content/development/extensions-core/postgresql.md new file mode 100644 index 000000000000..74ba86732861 --- /dev/null +++ b/docs/content/development/extensions-core/postgresql.md @@ -0,0 +1,49 @@ +--- +layout: doc_page +--- + +# PostgreSQL Metadata Store + +Make sure to [include](../../operations/including-extensions.html) `postgresql-metadata-storage` as an extension. + +## Setting up PostgreSQL + +1. Install PostgreSQL + + Use your favorite package manager to install PostgreSQL, e.g.: + - on Ubuntu/Debian using apt `apt-get install postgresql` + - on OS X, using [Homebrew](http://brew.sh/) `brew install postgresql` + +2. Create a druid database and user + + On the machine where PostgreSQL is installed, using an account with proper + postgresql permissions: + + Create a druid user, enter `diurd` when prompted for the password. + + ```bash + createuser druid -P + ``` + + Create a druid database owned by the user we just created + + ```bash + createdb druid -O druid + ``` + + *Note:* On Ubuntu / Debian you may have to prefix the `createuser` and + `createdb` commands with `sudo -u postgres` in order to gain proper + permissions. + +3. Configure your Druid metadata storage extension: + + Add the following parameters to your Druid configuration, replacing `` + with the location (host name and port) of the database. + + ```properties + druid.extensions.loadList=["postgresql-metadata-storage"] + druid.metadata.storage.type=postgresql + druid.metadata.storage.connector.connectURI=jdbc:postgresql:///druid + druid.metadata.storage.connector.user=druid + druid.metadata.storage.connector.password=diurd + ``` diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md new file mode 100644 index 000000000000..59f6a7084b3b --- /dev/null +++ b/docs/content/development/extensions-core/s3.md @@ -0,0 +1,38 @@ +--- +layout: doc_page +--- + +# S3-compatible + +Make sure to [include](../../operations/including-extensions.html) `druid-s3-extensions` as an extension. + +## Deep Storage + +S3-compatible deep storage is basically either S3 or something like Google Storage which exposes the same API as S3. + +### Configuration + +|Property|Possible Values|Description|Default| +|--------|---------------|-----------|-------| +|`druid.s3.accessKey`||S3 access key.|Must be set.| +|`druid.s3.secretKey`||S3 secret key.|Must be set.| +|`druid.storage.bucket`||Bucket to store in.|Must be set.| +|`druid.storage.baseKey`||Base key prefix to use, i.e. what directory.|Must be set.| + +## StaticS3Firehose + +This firehose ingests events from a predefined list of S3 objects. + +Sample spec: + +```json +"firehose" : { + "type" : "static-s3", + "uris": ["s3://foo/bar/file.gz", "s3://bar/foo/file2.gz"] +} +``` + +|property|description|default|required?| +|--------|-----------|-------|---------| +|type|This should be "static-s3"|N/A|yes| +|uris|JSON array of URIs where s3 files to be ingested are located.|N/A|yes| diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index 0598eb4c773b..a860b15dc054 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -21,16 +21,16 @@ Core extensions are maintained by Druid committers. |Name|Description|Docs| |----|-----------|----| -|druid-avro-extensions|Support for data in Apache Avro data format.|[link](../ingestion/index.html)| -|druid-datasketches|Support for approximate counts and set operations with [DataSketches](http://datasketches.github.io/).|[link](../development/datasketches-aggregators.html)| -|druid-hdfs-storage|Support for data in Apache Avro data format.|[link](../ingestion/index.html)| -|druid-histogram|HDFS deep storage.|[link](../dependencies/deep-storage.html#hdfs)| -|druid-kafka-eight|Kafka ingest firehose (high level consumer).|[link](../ingestion/firehose.html#kafkaeightfirehose)| -|druid-kafka-extraction-namespace|Kafka namespaced lookup.|[link](../querying/lookups.html#kafka-namespaced-lookup)| -|druid-namespace-lookup|Namespaced lookups.|[link](../querying/lookups.html)| -|druid-s3-extensions|S3 deep storage.|[link](../dependencies/deep-storage.html#s3-compatible/)| -|mysql-metadata-storage|MySQL metadata store.|[link](../dependencies/metadata-storage.html#setting-up-mysql)| -|postgresql-metadata-storage|PostgreSQL metadata store.|[link](../dependencies/metadata-storage.html#setting-up-postgresql)| +|druid-avro-extensions|Support for data in Apache Avro data format.|[link](../development/extensions-core/avro.html)| +|druid-datasketches|Support for approximate counts and set operations with [DataSketches](http://datasketches.github.io/).|[link](../development/extensions-core/datasketches-aggregators.html)| +|druid-hdfs-storage|HDFS deep storage.|[link](../development/extensions-core/hdfs.html)| +|druid-histogram|Approximate histograms and quantiles aggregator.|[link](../development/extensions-core/approximate-histograms.html)| +|druid-kafka-eight|Kafka ingest firehose (high level consumer).|[link](../development/extensions-core/kafka-eight-firehose.html)| +|druid-kafka-extraction-namespace|Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.html)| +|druid-namespace-lookup|Required module for [lookups](../querying/lookups.html).|[link](../development/extensions-core/namespaced-lookup.html)| +|druid-s3-extensions|Interfacing with data in AWS S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.html)| +|mysql-metadata-storage|MySQL metadata store.|[link](../development/extensions-core/mysql.html)| +|postgresql-metadata-storage|PostgreSQL metadata store.|[link](../development/extensions-core/postgresql.html)| # Community Extensions @@ -40,12 +40,12 @@ If you'd like to take on maintenance for a community extension, please post on [ |Name|Description|Docs| |----|-----------|----| -|druid-azure-extensions|Microsoft Azure deep storage.|[link](../development/community-extensions/azure.html)| -|druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/community-extensions/cassandra.html)| -|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/community-extensions/cloudfiles.html)| -|druid-kafka-eight-simpleConsumer|Kafka ingest firehose (low level consumer).|[link](../development/community-extensions/kafka-simple.html)| -|druid-rabbitmq|RabbitMQ firehose.|[link](../development/community-extensions/rabbitmq.html)| -|graphite-emitter|Graphite metrics emitter|[link](../development/community-extensions/graphite.html)| +|druid-azure-extensions|Microsoft Azure deep storage.|[link](../development/extensions-contrib/azure.html)| +|druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/extensions-contrib/cassandra.html)| +|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/extensions-contrib/cloudfiles.html)| +|druid-kafka-eight-simpleConsumer|Kafka ingest firehose (low level consumer).|[link](../development/extensions-contrib/kafka-simple.html)| +|druid-rabbitmq|RabbitMQ firehose.|[link](../development/extensions-contrib/rabbitmq.html)| +|graphite-emitter|Graphite metrics emitter|[link](../development/extensions-contrib/graphite.html)| ## Promoting Community Extension to Core Extension diff --git a/docs/content/ingestion/data-formats.md b/docs/content/ingestion/data-formats.md index afdeb8564ab5..7096fba75925 100644 --- a/docs/content/ingestion/data-formats.md +++ b/docs/content/ingestion/data-formats.md @@ -7,6 +7,8 @@ Data Formats for Ingestion Druid can ingest denormalized data in JSON, CSV, or a delimited form such as TSV, or any custom format. While most examples in the documentation use data in JSON format, it is not difficult to configure Druid to ingest any other delimited data. We welcome any contributions to new formats. +For additional data formats, please see our [extensions list](../development/extensions.html). + ## Formatting the Data The following are some samples of the data used in the [Wikipedia example](../tutorials/quickstart.html). diff --git a/docs/content/ingestion/firehose.md b/docs/content/ingestion/firehose.md index 12b33d59c28b..5b1c6a681981 100644 --- a/docs/content/ingestion/firehose.md +++ b/docs/content/ingestion/firehose.md @@ -4,67 +4,17 @@ layout: doc_page # Druid Firehoses -Firehoses describe the data stream source. They are pluggable and thus the configuration schema can and will vary based on the `type` of the firehose. +Firehoses are used in the [stream-pull](../ingestion/stream-pull.html) ingestion model. They are pluggable and thus the configuration schema can and will vary based on the `type` of the firehose. | Field | Type | Description | Required | |-------|------|-------------|----------| | type | String | Specifies the type of firehose. Each value will have its own configuration schema, firehoses packaged with Druid are described below. | yes | -We describe the configuration of the [Kafka firehose example](../ingestion/stream-pull.html#realtime-specfile), but there are other types available in Druid (see below). - -- `consumerProps` is a map of properties for the Kafka consumer. The JSON object is converted into a Properties object and passed along to the Kafka consumer. -- `feed` is the feed that the Kafka consumer should read from. - -## Available Firehoses +## Additional Firehoses There are several firehoses readily available in Druid, some are meant for examples, others can be used directly in a production environment. -#### KafkaEightFirehose - -Please note that the [druid-kafka-eight module](../operations/including-extensions.html) is required for this firehose. This firehose acts as a Kafka 0.8.x consumer and ingests data from Kafka. - -Sample spec: - -```json -"firehose": { - "type": "kafka-0.8", - "consumerProps": { - "zookeeper.connect": "localhost:2181", - "zookeeper.connection.timeout.ms" : "15000", - "zookeeper.session.timeout.ms" : "15000", - "zookeeper.sync.time.ms" : "5000", - "group.id": "druid-example", - "fetch.message.max.bytes" : "1048586", - "auto.offset.reset": "largest", - "auto.commit.enable": "false" - }, - "feed": "wikipedia" -} -``` - -|property|description|required?| -|--------|-----------|---------| -|type|This should be "kafka-0.8"|yes| -|consumerProps|The full list of consumer configs can be [here](https://kafka.apache.org/08/configuration.html).|yes| -|feed|Kafka maintains feeds of messages in categories called topics. This is the topic name.|yes| - -#### StaticS3Firehose - -This firehose ingests events from a predefined list of S3 objects. - -Sample spec: - -```json -"firehose" : { - "type" : "static-s3", - "uris": ["s3://foo/bar/file.gz", "s3://bar/foo/file2.gz"] -} -``` - -|property|description|default|required?| -|--------|-----------|-------|---------| -|type|This should be "static-s3"|N/A|yes| -|uris|JSON array of URIs where s3 files to be ingested are located.|N/A|yes| +For additional firehoses, please see our [extensions list](../development/extensions.html). #### LocalFirehose @@ -169,23 +119,3 @@ An example is shown below: |type|This should be "timed"|yes| |shutoffTime|time at which the firehose should shut down, in ISO8601 format|yes| |delegate|firehose to use|yes| - -#### TwitterSpritzerFirehose - -This firehose connects directly to the twitter spritzer data stream. - -Sample spec: - -```json -"firehose" : { - "type" : "twitzer", - "maxEventCount": -1, - "maxRunMinutes": 0 -} -``` - -|property|description|default|required?| -|--------|-----------|-------|---------| -|type|This should be "twitzer"|N/A|yes| -|maxEventCount|max events to receive, -1 is infinite, 0 means nothing is delivered; use this to prevent infinite space consumption or to prevent getting throttled at an inconvenient time.|N/A|yes| -|maxRunMinutes|maximum number of minutes to fetch Twitter events. Use this to prevent getting throttled at an inconvenient time. If zero or less, no time limit for run.|N/A|yes| diff --git a/docs/content/ingestion/index.md b/docs/content/ingestion/index.md index f681653fa6ea..63f2a62bf656 100644 --- a/docs/content/ingestion/index.md +++ b/docs/content/ingestion/index.md @@ -75,7 +75,7 @@ An example dataSchema is shown below: ## Parser -If `type` is not included, the parser defaults to `string`. +If `type` is not included, the parser defaults to `string`. For additional data formats, please see our [extensions list](../development/extensions.html). ### String Parser @@ -91,113 +91,6 @@ If `type` is not included, the parser defaults to `string`. | type | String | This should say `protobuf`. | no | | parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes | -### Avro Stream Parser - -This is for realtime ingestion. Make sure to include `druid-avro-extensions` as an extension. - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | This should say `avro_stream`. | no | -| avroBytesDecoder | JSON Object | Specifies how to decode bytes to Avro record. | yes | -| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes | - -For example, using Avro stream parser with schema repo Avro bytes decoder: - -```json -"parser" : { - "type" : "avro_stream", - "avroBytesDecoder" : { - "type" : "schema_repo", - "subjectAndIdConverter" : { - "type" : "avro_1124", - "topic" : "${YOUR_TOPIC}" - }, - "schemaRepository" : { - "type" : "avro_1124_rest_client", - "url" : "${YOUR_SCHEMA_REPO_END_POINT}", - } - }, - "parseSpec" : { - "type": "timeAndDims", - "timestampSpec": , - "dimensionsSpec": - } -} -``` - -#### Avro Bytes Decoder - -If `type` is not included, the avroBytesDecoder defaults to `schema_repo`. - -##### SchemaRepo Based Avro Bytes Decoder - -This Avro bytes decoder first extract `subject` and `id` from input message bytes, then use them to lookup the Avro schema with which to decode Avro record from bytes. Details can be found in [schema repo](https://github.com/schema-repo/schema-repo) and [AVRO-1124](https://issues.apache.org/jira/browse/AVRO-1124). You will need an http service like schema repo to hold the avro schema. Towards schema registration on the message producer side, you can refer to `io.druid.data.input.AvroStreamInputRowParserTest#testParse()`. - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | This should say `schema_repo`. | no | -| subjectAndIdConverter | JSON Object | Specifies the how to extract subject and id from message bytes. | yes | -| schemaRepository | JSON Object | Specifies the how to lookup Avro schema from subject and id. | yes | - -##### Avro-1124 Subject And Id Converter - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | This should say `avro_1124`. | no | -| topic | String | Specifies the topic of your kafka stream. | yes | - - -##### Avro-1124 Schema Repository - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | This should say `avro_1124_rest_client`. | no | -| url | String | Specifies the endpoint url of your Avro-1124 schema repository. | yes | - -### Avro Hadoop Parser - -This is for batch ingestion using the HadoopDruidIndexer. The `inputFormat` of `inputSpec` in `ioConfig` must be set to `"io.druid.data.input.avro.AvroValueInputFormat"`. You may want to set Avro reader's schema in `jobProperties` in `tuningConfig`, eg: `"avro.schema.path.input.value": "/path/to/your/schema.avsc"` or `"avro.schema.input.value": "your_schema_JSON_object"`, if reader's schema is not set, the schema in Avro object container file will be used, see [Avro specification](http://avro.apache.org/docs/1.7.7/spec.html#Schema+Resolution). Make sure to include "io.druid.extensions:druid-avro-extensions" as an extension. - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | This should say `avro_hadoop`. | no | -| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes | -| fromPigAvroStorage | Boolean | Specifies whether the data file is stored using AvroStorage. | no(default == false) | - -For example, using Avro Hadoop parser with custom reader's schema file: - -```json -{ - "type" : "index_hadoop", - "spec" : { - "dataSchema" : { - "dataSource" : "", - "parser" : { - "type" : "avro_hadoop", - "parseSpec" : { - "type": "timeAndDims", - "timestampSpec": , - "dimensionsSpec": - } - } - }, - "ioConfig" : { - "type" : "hadoop", - "inputSpec" : { - "type" : "static", - "inputFormat": "io.druid.data.input.avro.AvroValueInputFormat", - "paths" : "" - } - }, - "tuningConfig" : { - "jobProperties" : { - "avro.schema.path.input.value" : "/path/to/my/schema.avsc", - } - } - } -} -``` - ### ParseSpec ParseSpecs serve two purposes: diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index 185954f8452c..2fd5617c9278 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -1,6 +1,7 @@ --- layout: doc_page --- + # Lookups
@@ -27,272 +28,7 @@ and such data belongs in the raw denormalized data for use in Druid. Very small lookups (count of keys on the order of a few dozen to a few hundred) can be passed at query time as a "map" lookup as per [dimension specs](../querying/dimensionspecs.html). -Namespaced lookups are appropriate for lookups which are not possible to pass at query time due to their size, -or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers. -Namespaced lookups can be specified as part of the runtime properties file. The property is a list of the namespaces -described as per the sections on this page. For example: - - ```json - druid.query.extraction.namespace.lookups= - [ - { - "type": "uri", - "namespace": "some_uri_lookup", - "uri": "file:/tmp/prefix/", - "namespaceParseSpec": { - "format": "csv", - "columns": [ - "key", - "value" - ] - }, - "pollPeriod": "PT5M" - }, - { - "type": "jdbc", - "namespace": "some_jdbc_lookup", - "connectorConfig": { - "createTables": true, - "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", - "user": "druid", - "password": "diurd" - }, - "table": "lookupTable", - "keyColumn": "mykeyColumn", - "valueColumn": "MyValueColumn", - "tsColumn": "timeColumn" - } - ] - ``` - -Proper functionality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes: -`druid-namespace-lookup` - -## Cache Settings - -Lookups are cached locally on historical nodes. The following are settings used by the nodes which service queries when -setting namespaces (broker, peon, historical) - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.query.extraction.namespace.cache.type`|Specifies the type of caching to be used by the namespaces. May be one of [`offHeap`, `onHeap`]. `offHeap` uses a temporary file for off-heap storage of the namespace (memory mapped files). `onHeap` stores all cache on the heap in standard java map types.|`onHeap`| - -The cache is populated in different ways depending on the settings below. In general, most namespaces employ -a `pollPeriod` at the end of which time they poll the remote resource of interest for updates. A notable exception -is the Kafka namespace lookup, defined below. - -## URI namespace update - -The remapping values for each namespaced lookup can be specified by json as per - -```json -{ - "type":"uri", - "namespace":"some_lookup", - "uri": "s3://bucket/some/key/prefix/", - "namespaceParseSpec":{ - "format":"csv", - "columns":["key","value"] - }, - "pollPeriod":"PT5M", - "versionRegex": "renames-[0-9]*\\.gz" -} -``` - -|Property|Description|Required|Default| -|--------|-----------|--------|-------| -|`namespace`|The namespace to define|Yes|| -|`pollPeriod`|Period between polling for updates|No|0 (only once)| -|`versionRegex`|Regex to help find newer versions of the namespace data|Yes|| -|`namespaceParseSpec`|How to interpret the data at the URI|Yes|| - -The `pollPeriod` value specifies the period in ISO 8601 format between checks for updates. If the source of the lookup is capable of providing a timestamp, the lookup will only be updated if it has changed since the prior tick of `pollPeriod`. A value of 0, an absent parameter, or `null` all mean populate once and do not attempt to update. Whenever an update occurs, the updating system will look for a file with the most recent timestamp and assume that one with the most recent data. - -The `versionRegex` value specifies a regex to use to determine if a filename in the parent path of the uri should be considered when trying to find the latest version. Omitting this setting or setting it equal to `null` will match to all files it can find (equivalent to using `".*"`). The search occurs in the most significant "directory" of the uri. - -The `namespaceParseSpec` can be one of a number of values. Each of the examples below would rename foo to bar, baz to bat, and buck to truck. All parseSpec types assumes each input is delimited by a new line. See below for the types of parseSpec supported. - - -### csv lookupParseSpec - -|Parameter|Description|Required|Default| -|---------|-----------|--------|-------| -|`columns`|The list of columns in the csv file|yes|`null`| -|`keyColumn`|The name of the column containing the key|no|The first column| -|`valueColumn`|The name of the column containing the value|no|The second column| - -*example input* - -``` -bar,something,foo -bat,something2,baz -truck,something3,buck -``` - -*example namespaceParseSpec* - -```json -"namespaceParseSpec": { - "format": "csv", - "columns": ["value","somethingElse","key"], - "keyColumn": "key", - "valueColumn": "value" -} -``` - -### tsv lookupParseSpec - -|Parameter|Description|Required|Default| -|---------|-----------|--------|-------| -|`columns`|The list of columns in the csv file|yes|`null`| -|`keyColumn`|The name of the column containing the key|no|The first column| -|`valueColumn`|The name of the column containing the value|no|The second column| -|`delimiter`|The delimiter in the file|no|tab (`\t`)| - - -*example input* - -``` -bar|something,1|foo -bat|something,2|baz -truck|something,3|buck -``` - -*example namespaceParseSpec* - -```json -"namespaceParseSpec": { - "format": "tsv", - "columns": ["value","somethingElse","key"], - "keyColumn": "key", - "valueColumn": "value", - "delimiter": "|" -} -``` - -### customJson lookupParseSpec - -|Parameter|Description|Required|Default| -|---------|-----------|--------|-------| -|`keyFieldName`|The field name of the key|yes|null| -|`valueFieldName`|The field name of the value|yes|null| - -*example input* - -```json -{"key": "foo", "value": "bar", "somethingElse" : "something"} -{"key": "baz", "value": "bat", "somethingElse" : "something"} -{"key": "buck", "somethingElse": "something", "value": "truck"} -``` - -*example namespaceParseSpec* - -```json -"namespaceParseSpec": { - "format": "customJson", - "keyFieldName": "key", - "valueFieldName": "value" -} -``` - - -### simpleJson lookupParseSpec -The `simpleJson` lookupParseSpec does not take any parameters. It is simply a line delimited json file where the field is the key, and the field's value is the value. - -*example input* - -```json -{"foo": "bar"} -{"baz": "bat"} -{"buck": "truck"} -``` - -*example namespaceParseSpec* - -```json -"namespaceParseSpec":{ - "format": "simpleJson" -} -``` - -## JDBC namespaced lookup - -The JDBC lookups will poll a database to populate its local cache. If the `tsColumn` is set it must be able to accept comparisons in the format `'2015-01-01 00:00:00'`. For example, the following must be valid sql for the table `SELECT * FROM some_lookup_table WHERE timestamp_column > '2015-01-01 00:00:00'`. If `tsColumn` is set, the caching service will attempt to only poll values that were written *after* the last sync. If `tsColumn` is not set, the entire table is pulled every time. - -|Parameter|Description|Required|Default| -|---------|-----------|--------|-------| -|`namespace`|The namespace to define|Yes|| -|`connectorConfig`|The connector config to use|Yes|| -|`table`|The table which contains the key value pairs|Yes|| -|`keyColumn`|The column in `table` which contains the keys|Yes|| -|`valueColumn`|The column in `table` which contains the values|Yes|| -|`tsColumn`| The column in `table` which contains when the key was updated|No|Not used| -|`pollPeriod`|How often to poll the DB|No|0 (only once)| - -```json -{ - "type":"jdbc", - "namespace":"some_lookup", - "connectorConfig":{ - "createTables":true, - "connectURI":"jdbc:mysql://localhost:3306/druid", - "user":"druid", - "password":"diurd" - }, - "table":"some_lookup_table", - "keyColumn":"the_old_dim_value", - "valueColumn":"the_new_dim_value", - "tsColumn":"timestamp_column", - "pollPeriod":600000 -} -``` - -# Kafka namespaced lookup - -If you need updates to populate as promptly as possible, it is possible to plug into a kafka topic whose key is the old value and message is the desired new value (both in UTF-8). This requires the following extension: "io.druid.extensions:kafka-extraction-namespace" - -```json -{ - "type":"kafka", - "namespace":"testTopic", - "kafkaTopic":"testTopic" -} -``` - -|Parameter|Description|Required|Default| -|---------|-----------|--------|-------| -|`namespace`|The namespace to define|Yes|| -|`kafkaTopic`|The kafka topic to read the data from|Yes|| - - -## Kafka renames - -The extension `kafka-extraction-namespace` enables reading from a kafka feed which has name/key pairs to allow renaming of dimension values. An example use case would be to rename an ID to a human readable format. - -Currently the historical node caches the key/value pairs from the kafka feed in an ephemeral memory mapped DB via MapDB. - -## Configuration - -The following options are used to define the behavior and should be included wherever the extension is included (all query servicing nodes): - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.query.rename.kafka.properties`|A json map of kafka consumer properties. See below for special properties.|See below| - -The following are the handling for kafka consumer properties in `druid.query.rename.kafka.properties` - -|Property|Description|Default| -|--------|-----------|-------| -|`zookeeper.connect`|Zookeeper connection string|`localhost:2181/kafka`| -|`group.id`|Group ID, auto-assigned for publish-subscribe model and cannot be overridden|`UUID.randomUUID().toString()`| -|`auto.offset.reset`|Setting to get the entire kafka rename stream. Cannot be overridden|`smallest`| - -## Testing the Kafka rename functionality - -To test this setup, you can send key/value pairs to a kafka stream via the following producer console: - -``` -./bin/kafka-console-producer.sh --property parse.key=true --property key.separator="->" --broker-list localhost:9092 --topic testTopic -``` +For static lookups defined in `runtime.properties` rather than embedded in the query, please look at +the experimental [namespaced lookup extension](../development/extensions-core/namespaced-lookup.html). -Renames can then be published as `OLD_VAL->NEW_VAL` followed by newline (enter or return) +For other additional lookups, please see our [extensions list](../development/extensions.html). diff --git a/docs/content/toc.md b/docs/content/toc.md index bb0582c0e9c0..ce701362bb71 100644 --- a/docs/content/toc.md +++ b/docs/content/toc.md @@ -93,9 +93,9 @@ * [Integration](../development/integrating-druid-with-other-technologies.html) * Experimental Features * [Overview](../development/experimental.html) + * [Approximate Histograms and Quantiles](../development/extensions-core/approximate-histograms.html) + * [Datasketches](../development/extensions-core/datasketches-aggregators.html) * [Geographic Queries](../development/geo.html) - * [Approximate Histograms and Quantiles](../development/approximate-histograms.html) - * [Datasketches](../development/datasketches-aggregators.html) * [Router](../development/router.html) ## Misc diff --git a/extensions-contrib/README.md b/extensions-contrib/README.md index 943b30a67595..0526043c5f3d 100644 --- a/extensions-contrib/README.md +++ b/extensions-contrib/README.md @@ -1,6 +1,6 @@ # Community Extensions -Please contribute all community extensions in this directory and include a doc of how your extension can be used under /docs/content/development/community-extensions/. +Please contribute all community extensions in this directory and include a doc of how your extension can be used under /docs/content/development/extensions-contrib/. Please note that community extensions are maintained by their original contributors and are not packaged with the core Druid distribution. If you'd like to take on maintenance for a community extension, please post on [druid-development group](https://groups.google.com/forum/#!forum/druid-development) to let us know! From b9c2f0c0fa2fd40a30be0dac81f8e045b04a4fde Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 22 Mar 2016 13:59:57 -0700 Subject: [PATCH 3/3] Fix a bunch of broken links in the docs. (Backport of #2699) --- docs/content/dependencies/metadata-storage.md | 12 ++++++------ docs/content/ingestion/batch-ingestion.md | 4 ++-- docs/content/ingestion/stream-pull.md | 3 --- docs/content/ingestion/update-existing-data.md | 2 +- docs/content/querying/dimensionspecs.md | 2 +- docs/content/tutorials/cluster.md | 6 +++--- docs/content/tutorials/quickstart.md | 4 ++-- docs/content/tutorials/tutorial-batch.md | 2 +- docs/content/tutorials/tutorial-kafka.md | 2 +- 9 files changed, 17 insertions(+), 20 deletions(-) diff --git a/docs/content/dependencies/metadata-storage.md b/docs/content/dependencies/metadata-storage.md index ddf8855e7b7c..d847bbc1d286 100644 --- a/docs/content/dependencies/metadata-storage.md +++ b/docs/content/dependencies/metadata-storage.md @@ -17,13 +17,13 @@ Derby is not suitable for production use as a metadata store. Use MySQL or Postg ## Using derby - Add the following to your Druid configuration. +Add the following to your Druid configuration. + +```properties +druid.metadata.storage.type=derby +druid.metadata.storage.connector.connectURI=jdbc:derby://localhost:1527//opt/var/druid_state/derby;create=true +``` - ```properties - druid.metadata.storage.type=derby - druid.metadata.storage.connector.connectURI=jdbc:derby://localhost:1527//home/y/var/druid_state/derby;create=true - ``` - ## MySQL See [mysql-metadata-storage extension documentation](../development/extensions-core/mysql.html). diff --git a/docs/content/ingestion/batch-ingestion.md b/docs/content/ingestion/batch-ingestion.md index b8cc8ccb9127..f713efdb176c 100644 --- a/docs/content/ingestion/batch-ingestion.md +++ b/docs/content/ingestion/batch-ingestion.md @@ -238,7 +238,7 @@ classification=yarn-site,properties=[mapreduce.reduce.memory.mb=6144,mapreduce.r ``` - Follow the instructions under "[Configure Hadoop for data -loads](cluster.html#configure-cluster-for-hadoop-data-loads)" using the XML files from +loads](../tutorials/cluster.html#configure-cluster-for-hadoop-data-loads)" using the XML files from `/etc/hadoop/conf` on your EMR master. #### Loading from S3 with EMR @@ -268,7 +268,7 @@ Druid works out of the box with many Hadoop distributions. If you are having dependency conflicts between Druid and your version of Hadoop, you can try searching for a solution in the [Druid user groups](https://groups.google.com/forum/#!forum/druid- -user), or reading the Druid [Different Hadoop Versions](..//operations/other-hadoop.html) documentation. +user), or reading the Druid [Different Hadoop Versions](../operations/other-hadoop.html) documentation. ## Command Line Hadoop Indexer diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index e43b15a14af2..42201f073517 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -292,9 +292,6 @@ results. Is this always a problem? No. If your data is small enough to fit on a single Kafka partition, you can replicate without issues. Otherwise, you can run real-time nodes without replication. -There is now also an [experimental low level Kafka firehose](../development/kafka-simple-consumer-firehose.html) which -solves the issues described above with using the high level Kafka consumer. - ### Locking Using stream pull ingestion with Realtime nodes together batch ingestion may introduce data override issues. For example, if you diff --git a/docs/content/ingestion/update-existing-data.md b/docs/content/ingestion/update-existing-data.md index c8284e8d0caa..b989ea4ae148 100644 --- a/docs/content/ingestion/update-existing-data.md +++ b/docs/content/ingestion/update-existing-data.md @@ -28,7 +28,7 @@ segments and avoid the overhead of rebuilding new segments with reindexing, you ### Reindexing and Delta Ingestion with Hadoop Batch Ingestion This section assumes the reader understands how to do batch ingestion using Hadoop. See -[batch-ingestion](batch-ingestion.md) for more information. Hadoop batch-ingestion can be used for reindexing and delta ingestion. +[batch-ingestion](batch-ingestion.html) for more information. Hadoop batch-ingestion can be used for reindexing and delta ingestion. Druid uses an `inputSpec` in the `ioConfig` to know where the data to be ingested is located and how to read it. For simple Hadoop batch ingestion, `static` or `granularity` spec types allow you to read data stored in deep storage. diff --git a/docs/content/querying/dimensionspecs.md b/docs/content/querying/dimensionspecs.md index ab6185daa851..2150c01745cd 100644 --- a/docs/content/querying/dimensionspecs.md +++ b/docs/content/querying/dimensionspecs.md @@ -353,7 +353,7 @@ For example if you want to concat "[" and "]" before and after the actual dimens ### Filtered DimensionSpecs -These are only valid for multi-value dimensions. If you have a row in druid that has a multi-value dimension with values ["v1", "v2", "v3"] and you send a groupBy/topN query grouping by that dimension with [query filter](filter.html) for value "v1". In the response you will get 3 rows containing "v1", "v2" and "v3". This behavior might be unintuitive for some use cases. +These are only valid for multi-value dimensions. If you have a row in druid that has a multi-value dimension with values ["v1", "v2", "v3"] and you send a groupBy/topN query grouping by that dimension with [query filter](filters.html) for value "v1". In the response you will get 3 rows containing "v1", "v2" and "v3". This behavior might be unintuitive for some use cases. It happens because "query filter" is internally used on the bitmaps and only used to match the row to be included in the query result processing. With multi-value dimensions, "query filter" behaves like a contains check, which will match the row with dimension value ["v1", "v2", "v3"]. Please see the section on "Multi-value columns" in [segment](../design/segments.html) for more details. Then groupBy/topN processing pipeline "explodes" all multi-value dimensions resulting 3 rows for "v1", "v2" and "v3" each. diff --git a/docs/content/tutorials/cluster.md b/docs/content/tutorials/cluster.md index b213eaa34d22..697d98cc56db 100644 --- a/docs/content/tutorials/cluster.md +++ b/docs/content/tutorials/cluster.md @@ -68,13 +68,13 @@ In this package, you'll find: * `LICENSE` - the license files. -* `bin/` - scripts related to the [single-machine quickstart](quickstart.md). +* `bin/` - scripts related to the [single-machine quickstart](quickstart.html). * `conf/*` - template configurations for a clustered setup. -* `conf-quickstart/*` - configurations for the [single-machine quickstart](quickstart.md). +* `conf-quickstart/*` - configurations for the [single-machine quickstart](quickstart.html). * `extensions/*` - all Druid extensions. * `hadoop-dependencies/*` - Druid Hadoop dependencies. * `lib/*` - all included software packages for core Druid. -* `quickstart/*` - files related to the [single-machine quickstart](quickstart.md). +* `quickstart/*` - files related to the [single-machine quickstart](quickstart.html). We'll be editing the files in `conf/` in order to get things running. diff --git a/docs/content/tutorials/quickstart.md b/docs/content/tutorials/quickstart.md index d0bcb3bd08c3..2223c7ae7b8a 100644 --- a/docs/content/tutorials/quickstart.md +++ b/docs/content/tutorials/quickstart.md @@ -174,7 +174,7 @@ bin/tranquility server -configFile /conf-quickstart/tranqu
This section shows you how to load data using Tranquility Server, but Druid also supports a wide -variety of other streaming ingestion options, including from +variety of other streaming ingestion options, including from popular streaming systems like Kafka, Storm, Samza, and Spark Streaming.
@@ -229,7 +229,7 @@ visualize and explore data in Druid. We recommend trying [Pivot](https://github. [Panoramix](https://github.com/mistercrunch/panoramix), or [Metabase](https://github.com/metabase/metabase) to start visualizing the data you just ingested. -If you installed Pivot for example, you should be able to view your data in your browser at [localhost:9090](localhost:9090). +If you installed Pivot for example, you should be able to view your data in your browser at [localhost:9090](http://localhost:9090/). ### SQL and other query libraries diff --git a/docs/content/tutorials/tutorial-batch.md b/docs/content/tutorials/tutorial-batch.md index cc4d9f3e6218..3d14a9a84491 100644 --- a/docs/content/tutorials/tutorial-batch.md +++ b/docs/content/tutorials/tutorial-batch.md @@ -16,7 +16,7 @@ Once that's complete, you can load your own dataset by writing a custom ingestio ## Writing an ingestion spec -When loading files into Druid, you will use Druid's [batch loading](ingestion-batch.html) process. +When loading files into Druid, you will use Druid's [batch loading](../ingestion/batch-ingestion.html) process. There's an example batch ingestion spec in `quickstart/wikiticker-index.json` that you can modify for your own needs. diff --git a/docs/content/tutorials/tutorial-kafka.md b/docs/content/tutorials/tutorial-kafka.md index eddd927047bb..ceb741c7e54f 100644 --- a/docs/content/tutorials/tutorial-kafka.md +++ b/docs/content/tutorials/tutorial-kafka.md @@ -45,7 +45,7 @@ Run this command to create a Kafka topic called *metrics*, to which we'll send d ## Enable Druid Kafka ingestion -Druid includes configs for [Tranquility Kafka](ingestion-streams.md#kafka) to support loading data from Kafka. +Druid includes configs for [Tranquility Kafka](../ingestion/stream-pull.html#kafka) to support loading data from Kafka. To enable this in the quickstart-based configuration: - Stop your Tranquility command (CTRL-C) and then start it up again.