diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index c8fec800e3e..7cc61e1744d 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -18,55 +18,68 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 - io.druid.extensions - druid-cassandra-storage - druid-cassandra-storage - druid-cassandra-storage + + 4.0.0 + io.druid.extensions + druid-cassandra-storage + druid-cassandra-storage + druid-cassandra-storage - - io.druid - druid - 0.7.0-SNAPSHOT - + + io.druid + druid + 0.7.0-SNAPSHOT + - - - io.druid - druid-api - - - com.netflix.astyanax - astyanax - 1.0.1 - - - commons-io - commons-io - + + + io.druid + druid-api + + + com.netflix.astyanax + astyanax + 1.0.1 + + + commons-io + commons-io + - - - junit - junit - test - - + + + junit + junit + test + + - - - - maven-jar-plugin - - - - true - true - - - - - - + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + true + true + + + + + + diff --git a/common/pom.xml b/common/pom.xml index d54a3486cbf..1020eabc6d8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -18,142 +18,155 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 + + 4.0.0 + io.druid + druid-common + druid-common + druid-common + + io.druid - druid-common - druid-common - druid-common + druid + 0.7.0-SNAPSHOT + - - io.druid - druid - 0.7.0-SNAPSHOT - + + + com.metamx + java-util + + + io.druid + druid-api + - - - com.metamx - java-util - - - io.druid - druid-api - + + commons-codec + commons-codec + + + org.apache.commons + commons-dbcp2 + + + commons-pool + commons-pool + 1.6 + + + org.skife.config + config-magic + + + org.hibernate + hibernate-validator + + + javax.validation + validation-api + + + com.google.guava + guava + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.datatype + jackson-datatype-guava + + + com.fasterxml.jackson.datatype + jackson-datatype-joda + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + + + com.google.inject + guice + + + com.google.inject.extensions + guice-multibindings + + + org.jdbi + jdbi + + + joda-time + joda-time + + + com.google.code.findbugs + jsr305 + + + log4j + log4j + - - commons-codec - commons-codec - - - org.apache.commons - commons-dbcp2 - - - commons-pool - commons-pool - 1.6 - - - org.skife.config - config-magic - - - org.hibernate - hibernate-validator - - - javax.validation - validation-api - - - com.google.guava - guava - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.datatype - jackson-datatype-guava - - - com.fasterxml.jackson.datatype - jackson-datatype-joda - - - com.fasterxml.jackson.dataformat - jackson-dataformat-smile - - - com.google.inject - guice - - - com.google.inject.extensions - guice-multibindings - - - org.jdbi - jdbi - - - joda-time - joda-time - - - com.google.code.findbugs - jsr305 - - - log4j - log4j - + + + junit + junit + test + + + org.easymock + easymock + test + + + com.metamx + java-util + test-jar + test + + - - - junit - junit - test - - - org.easymock - easymock - test - - - com.metamx - java-util - test-jar - test - - - - - - - maven-jar-plugin - - - - test-jar - - - - - - - true - true - - - - - - + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + test-jar + + + + + + + true + true + + + + + + diff --git a/common/src/main/java/io/druid/common/utils/SerializerUtils.java b/common/src/main/java/io/druid/common/utils/SerializerUtils.java index 1f6edb43f4e..fff67a9c339 100644 --- a/common/src/main/java/io/druid/common/utils/SerializerUtils.java +++ b/common/src/main/java/io/druid/common/utils/SerializerUtils.java @@ -21,6 +21,7 @@ package io.druid.common.utils; import com.google.common.io.ByteStreams; import com.google.common.io.OutputSupplier; +import com.google.common.primitives.Ints; import io.druid.collections.IntList; import java.io.IOException; @@ -262,4 +263,9 @@ public class SerializerUtils return retVal; } + + public int getSerializedStringByteSize(String str) + { + return Ints.BYTES + str.getBytes(UTF8).length; + } } diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageConnectorConfig.java b/common/src/main/java/io/druid/metadata/MetadataStorageConnectorConfig.java index 07447c36a74..048cc0e99e3 100644 --- a/common/src/main/java/io/druid/metadata/MetadataStorageConnectorConfig.java +++ b/common/src/main/java/io/druid/metadata/MetadataStorageConnectorConfig.java @@ -29,7 +29,13 @@ public class MetadataStorageConnectorConfig private boolean createTables = true; @JsonProperty - private String connectURI = null; + private String host = "localhost"; + + @JsonProperty + private int port = 1527; + + @JsonProperty + private String connectURI; @JsonProperty private String user = null; @@ -37,16 +43,26 @@ public class MetadataStorageConnectorConfig @JsonProperty private String password = null; - @JsonProperty - private String validationQuery = "SELECT 1"; - public boolean isCreateTables() { return createTables; } + public String getHost() + { + return host; + } + + public int getPort() + { + return port; + } + public String getConnectURI() { + if (connectURI == null) { + return String.format("jdbc:derby://%s:%s/druid;create=true", host, port); + } return connectURI; } @@ -60,20 +76,14 @@ public class MetadataStorageConnectorConfig return password; } - public String getValidationQuery() - { - return validationQuery; - } - @Override public String toString() { return "DbConnectorConfig{" + "createTables=" + createTables + - ", connectURI='" + connectURI + '\'' + + ", connectURI='" + getConnectURI() + '\'' + ", user='" + user + '\'' + ", password=****" + - ", validationQuery='" + validationQuery + '\'' + '}'; } } diff --git a/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java b/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java index 7d7f80e2bfe..3931196b17e 100644 --- a/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java +++ b/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java @@ -20,6 +20,7 @@ package io.druid.collections; import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.metamx.common.guava.BaseSequence; @@ -28,7 +29,7 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.SequenceTestHelper; import com.metamx.common.guava.Sequences; import com.metamx.common.guava.TestSequence; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; @@ -59,6 +60,61 @@ public class OrderedMergeSequenceTest } } + @Test + public void testMergeEmptySequence() throws Exception + { + final ArrayList> testSequences = Lists.newArrayList( + TestSequence.create(ImmutableList.of()), + TestSequence.create(2, 8), + TestSequence.create(4, 6, 8) + ); + + OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); + + SequenceTestHelper.testAll(seq, Arrays.asList(2, 4, 6, 8, 8)); + + for (TestSequence sequence : testSequences) { + Assert.assertTrue(sequence.isClosed()); + } + } + + @Test + public void testMergeEmptySequenceAtEnd() throws Exception + { + final ArrayList> testSequences = Lists.newArrayList( + TestSequence.create(2, 8), + TestSequence.create(4, 6, 8), + TestSequence.create(ImmutableList.of()) + ); + + OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); + + SequenceTestHelper.testAll(seq, Arrays.asList(2, 4, 6, 8, 8)); + + for (TestSequence sequence : testSequences) { + Assert.assertTrue(sequence.isClosed()); + } + } + + + @Test + public void testMergeEmptySequenceMiddle() throws Exception + { + final ArrayList> testSequences = Lists.newArrayList( + TestSequence.create(2, 8), + TestSequence.create(ImmutableList.of()), + TestSequence.create(4, 6, 8) + ); + + OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); + + SequenceTestHelper.testAll(seq, Arrays.asList(2, 4, 6, 8, 8)); + + for (TestSequence sequence : testSequences) { + Assert.assertTrue(sequence.isClosed()); + } + } + @Test public void testScrewsUpOnOutOfOrderBeginningOfList() throws Exception { diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index 9aae64dcf51..91beca02f25 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -82,7 +82,7 @@ Data segment announcers are used to announce segments. |Property|Description|Default| |--------|-----------|-------| -|`druid.announcer.type`|Choices: legacy or batch. The type of data segment announcer to use.|legacy| +|`druid.announcer.type`|Choices: legacy or batch. The type of data segment announcer to use.|batch| ##### Single Data Segment Announcer diff --git a/docs/content/DataSource.md b/docs/content/DataSource.md index 49c583561ba..bfbe1c6b4d9 100644 --- a/docs/content/DataSource.md +++ b/docs/content/DataSource.md @@ -1,18 +1,34 @@ --- layout: doc_page --- -A data source is the Druid equivalent of a database table. However, a query can also masquerade as a data source, providing subquery-like functionality. Query data sources are currently only supported by [GroupBy](GroupByQuery.html) queries. + +## Datasources + +A data source is the Druid equivalent of a database table. However, a query can also masquerade as a data source, providing subquery-like functionality. Query data sources are currently supported only by [GroupBy](GroupByQuery.html) queries. ### Table Data Source -The table data source the most common type. It's represented by a string, or by the full structure: +The table data source is the most common type. It's represented by a string, or by the full structure: ```json { "type": "table", - "name": + "name": "" } ``` +### Union Data Source +This data source unions two or more table data sources. + +```json +{ + "type": "union", + "dataSources": ["", "", "", ... ] +} +``` + +Note that the data sources being unioned should have the same schema. + + ### Query Data Source ```json { diff --git a/docs/content/DimensionSpecs.md b/docs/content/DimensionSpecs.md index c9cb351343e..690324df456 100644 --- a/docs/content/DimensionSpecs.md +++ b/docs/content/DimensionSpecs.md @@ -29,9 +29,9 @@ Returns dimension values transformed using the given [DimExtractionFn](#toc_3) } ``` -## DimExtractionFn +## DimExtractionFn -`DimExtractionFn`s define the transformation applied to each dimenion value +`DimExtractionFn`s define the transformation applied to each dimension value ### RegexDimExtractionFn diff --git a/docs/content/Firehose.md b/docs/content/Firehose.md index f2c5d9cfa93..fe2fd6a5092 100644 --- a/docs/content/Firehose.md +++ b/docs/content/Firehose.md @@ -93,6 +93,22 @@ A sample ingest firehose spec is shown below - |metrics|The list of metrics to select. If left empty, no metrics are returned. If left null or not defined, all metrics are selected.|no| |filter| See [Filters](Filters.html)|yes| +#### CombiningFirehose +This firehose can be used to combine and merge data from a list of different firehoses. +This can be used to merge data from more than one firehoses. + +```json +{ + "type" : "combining", + "delegates" : [ { firehose1 }, { firehose2 }, ..... ] +} +``` + +|property|description|required?| +|--------|-----------|---------| +|type|combining|yes| +|delegates|list of firehoses to combine data from|yes| + Parsing Data ------------ diff --git a/docs/content/Granularities.md b/docs/content/Granularities.md index e8a3e4fc2bf..162d22bf5c4 100644 --- a/docs/content/Granularities.md +++ b/docs/content/Granularities.md @@ -17,25 +17,25 @@ Supported granularity strings are: `all`, `none`, `minute`, `fifteen_minute`, `t ### Duration Granularities -Duration granularities are specified as an exact duration in milliseconds and timestamps are returned as UTC. +Duration granularities are specified as an exact duration in milliseconds and timestamps are returned as UTC. Duration granularity values are in millis. They also support specifying an optional origin, which defines where to start counting time buckets from (defaults to 1970-01-01T00:00:00Z). ```javascript -{"type": "duration", "duration": "7200000"} +{"type": "duration", "duration": 7200000} ``` This chunks up every 2 hours. ```javascript -{"type": "duration", "duration": "3600000", "origin": "2012-01-01T00:30:00Z"} +{"type": "duration", "duration": 3600000, "origin": "2012-01-01T00:30:00Z"} ``` This chunks up every hour on the half-hour. ### Period Granularities -Period granularities are specified as arbitrary period combinations of years, months, weeks, hours, minutes and seconds (e.g. P2W, P3M, PT1H30M, PT0.750S) in ISO8601 format. They support specifying a time zone which determines where period boundaries start as well as the timezone of the returned timestamps. By default, years start on the first of January, months start on the first of the month and weeks start on Mondays unless an origin is specified. +Period granularities are specified as arbitrary period combinations of years, months, weeks, hours, minutes and seconds (e.g. P2W, P3M, PT1H30M, PT0.750S) in [ISO8601](https://en.wikipedia.org/wiki/ISO_8601) format. They support specifying a time zone which determines where period boundaries start as well as the timezone of the returned timestamps. By default, years start on the first of January, months start on the first of the month and weeks start on Mondays unless an origin is specified. Time zone is optional (defaults to UTC). Origin is optional (defaults to 1970-01-01T00:00:00 in the given time zone). diff --git a/docs/content/GroupByQuery.md b/docs/content/GroupByQuery.md index 0c6cf0cdf67..c010d8daf6c 100644 --- a/docs/content/GroupByQuery.md +++ b/docs/content/GroupByQuery.md @@ -49,7 +49,7 @@ There are 11 main parts to a groupBy query: |--------|-----------|---------| |queryType|This String should always be "groupBy"; this is the first thing Druid looks at to figure out how to interpret the query|yes| |dataSource|A String defining the data source to query, very similar to a table in a relational database, or a [DataSource](DataSource.html) structure.|yes| -|dimensions|A JSON list of dimensions to do the groupBy over|yes| +|dimensions|A JSON list of dimensions to do the groupBy over; or see [DimensionSpec](DimensionSpecs) for ways to extract dimensions. |yes| |limitSpec|See [LimitSpec](LimitSpec.html).|no| |having|See [Having](Having.html).|no| |granularity|Defines the granularity of the query. See [Granularities](Granularities.html)|yes| diff --git a/docs/content/Having.md b/docs/content/Having.md index e3528be48b7..e54c74ec863 100644 --- a/docs/content/Having.md +++ b/docs/content/Having.md @@ -13,6 +13,15 @@ Druid supports the following types of having clauses. The simplest having clause is a numeric filter. Numeric filters can be used as the base filters for more complex boolean expressions of filters. +Here's an example of a having-clause numeric filter: + +```json +{ + "type": "greaterThan", + "aggregation": "myAggMetric", + "value": 100 +} + #### Equal To The equalTo filter will match rows with a specific aggregate value. @@ -21,7 +30,7 @@ The grammar for an `equalTo` filter is as follows: ```json { "type": "equalTo", - "aggregation": , + "aggregation": "", "value": } ``` @@ -36,7 +45,7 @@ The grammar for a `greaterThan` filter is as follows: ```json { "type": "greaterThan", - "aggregation": , + "aggregation": "", "value": } ``` @@ -51,7 +60,7 @@ The grammar for a `greaterThan` filter is as follows: ```json { "type": "lessThan", - "aggregation": , + "aggregation": "", "value": } ``` diff --git a/docs/content/LimitSpec.md b/docs/content/LimitSpec.md index e2ad6c91210..a3f7a487f3f 100644 --- a/docs/content/LimitSpec.md +++ b/docs/content/LimitSpec.md @@ -18,11 +18,13 @@ The default limit spec takes a limit and the list of columns to do an orderBy op #### OrderByColumnSpec -OrderByColumnSpecs indicate how to do order by operations. Each order by condition can be a `jsonString` or a map of the following form: +OrderByColumnSpecs indicate how to do order by operations. Each order-by condition can be a `jsonString` or a map of the following form: ```json { - "dimension" : , - "direction" : "ASCENDING OR DESCENDING" + "dimension" : "", + "direction" : <"ascending"|"descending"> } ``` + +If only the dimension is provided (as a JSON string), the default order-by is ascending. diff --git a/docs/content/Modules.md b/docs/content/Modules.md index b5b8a693053..229d6115c4a 100644 --- a/docs/content/Modules.md +++ b/docs/content/Modules.md @@ -4,7 +4,7 @@ layout: doc_page # Extending Druid With Custom Modules -Druid version 0.6 introduces a new module system that allows for the addition of extensions at runtime. +Druid uses a module system that allows for the addition of extensions at runtime. ## Specifying extensions @@ -20,11 +20,12 @@ Druid has the ability to automatically load extension jars from maven at runtime ## Configuring the extensions -Druid 0.6 introduces four new properties for configuring the loading of extensions: - +Druid provides the following settings to configure the loading of extensions: * `druid.extensions.coordinates` - This is a JSON Array list of "groupId:artifactId:version" maven coordinates. Defaults to `[]` + This is a JSON array of "groupId:artifactId[:version]" maven coordinates. For artifacts without version specified, Druid will append the default version. Defaults to `[]` +* `druid.extensions.defaultVersion` + Version to use for extension artifacts without version information. Defaults to the `druid-server` artifact version. * `druid.extensions.localRepository` This specifies where to look for the "local repository". The way maven gets dependencies is that it downloads them to a "local repository" on your local disk and then collects the paths to each of the jars. This specifies the directory to consider the "local repository". Defaults to `~/.m2/repository` * `druid.extensions.remoteRepositories` diff --git a/docs/content/Router.md b/docs/content/Router.md index 064bf1d646d..d23af149fec 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -119,15 +119,15 @@ Including this strategy means all timeBoundary queries are always routed to the Queries with a priority set to less than minPriority are routed to the lowest priority broker. Queries with priority set to greater than maxPriority are routed to the highest priority broker. By default, minPriority is 0 and maxPriority is 1. Using these default values, if a query with priority 0 (the default query priority is 0) is sent, the query skips the priority selection logic. -### javascript +### JavaScript Allows defining arbitrary routing rules using a JavaScript function. The function is passed the configuration and the query to be executed, and returns the tier it should be routed to, or null for the default tier. -*Example*: a function that return the highest priority broker unless the given query has more than two aggregators. +*Example*: a function that sends queries containing more than three aggregators to the lowest priority broker. ```json { "type" : "javascript", - "function" : "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" + "function" : "function (config, query) { if (query.getAggregatorSpecs && query.getAggregatorSpecs().size() >= 3) { var size = config.getTierToBrokerMap().values().size(); if (size > 0) { return config.getTierToBrokerMap().values().toArray()[size-1] } else { return config.getDefaultBrokerServiceName() } } else { return null } }" } ``` diff --git a/docs/content/Rule-Configuration.md b/docs/content/Rule-Configuration.md index c25c9e62b70..c0a310ad041 100644 --- a/docs/content/Rule-Configuration.md +++ b/docs/content/Rule-Configuration.md @@ -27,7 +27,7 @@ Forever load rules are of the form: } ``` -* `type` - this should always be "loadByInterval" +* `type` - this should always be "loadForever" * `tieredReplicants` - A JSON Object where the keys are the tier names and values are the number of replicas for that tier. @@ -86,7 +86,7 @@ Forever drop rules are of the form: } ``` -* `type` - this should always be "dropByPeriod" +* `type` - this should always be "dropForever" All segments that match this rule are dropped from the cluster. diff --git a/docs/content/SearchQuery.md b/docs/content/SearchQuery.md index 804b64532ab..b2ee5f3cf61 100644 --- a/docs/content/SearchQuery.md +++ b/docs/content/SearchQuery.md @@ -30,14 +30,14 @@ There are several main parts to a search query: |property|description|required?| |--------|-----------|---------| -|queryType|This String should always be "search"; this is the first thing Druid looks at to figure out how to interpret the query|yes| -|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| -|granularity|Defines the granularity of the query. See [Granularities](Granularities.html)|yes| -|filter|See [Filters](Filters.html)|no| +|queryType|This String should always be "search"; this is the first thing Druid looks at to figure out how to interpret the query.|yes| +|dataSource|A String defining the data source to query, very similar to a table in a relational database.|yes| +|granularity|Defines the granularity of the query. See [Granularities](Granularities.html).|yes| +|filter|See [Filters](Filters.html).|no| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |searchDimensions|The dimensions to run the search over. Excluding this means the search is run over all dimensions.|no| |query|See [SearchQuerySpec](SearchQuerySpec.html).|yes| -|sort|How the results of the search should be sorted. Two possible types here are "lexicographic" and "strlen".|yes| +|sort|An object specifying how the results of the search should be sorted. Two possible types here are "lexicographic" (the default sort) and "strlen".|no| |context|An additional JSON Object which can be used to specify certain flags.|no| The format of the result is: diff --git a/docs/content/SelectQuery.md b/docs/content/SelectQuery.md index 9149358d605..caeb9e0a162 100644 --- a/docs/content/SelectQuery.md +++ b/docs/content/SelectQuery.md @@ -26,9 +26,9 @@ There are several main parts to a select query: |dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |filter|See [Filters](Filters.html)|no| -|dimensions|The list of dimensions to select. If left empty, all dimensions are returned.|no| -|metrics|The list of metrics to select. If left empty, all metrics are returned.|no| -|pagingSpec|A JSON object indicating offsets into different scanned segments. Select query results will return a pagingSpec that can be reused for pagination.|yes| +|dimensions|A String array of dimensions to select. If left empty, all dimensions are returned.|no| +|metrics|A String array of metrics to select. If left empty, all metrics are returned.|no| +|pagingSpec|A JSON object indicating offsets into different scanned segments. Query results will return a `pagingIdentifiers` value that can be reused in the next query for pagination.|yes| |context|An additional JSON Object which can be used to specify certain flags.|no| The format of the result is: @@ -140,4 +140,30 @@ The format of the result is: } ] ``` -The result returns a global pagingSpec that can be reused for the next select query. The offset will need to be increased by 1 on the client side. \ No newline at end of file +The `threshold` determines how many hits are returned, with each hit indexed by an offset. + +The results above include: + +```json + "pagingIdentifiers" : { + "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" : 4 + }, +``` + +This can be used with the next query's pagingSpec: + +```json + { + "queryType": "select", + "dataSource": "wikipedia", + "dimensions":[], + "metrics":[], + "granularity": "all", + "intervals": [ + "2013-01-01/2013-01-02" + ], + "pagingSpec":{"pagingIdentifiers": {"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" : 5}, "threshold":5} + + } + +Note that in the second query, an offset is specified and that it is 1 greater than the largest offset found in the initial results. To return the next "page", this offset must be incremented by 1 with each new query. When an empty results set is received, the very last page has been returned. diff --git a/docs/content/TopNMetricSpec.md b/docs/content/TopNMetricSpec.md index 10f5f33091d..978c80b5005 100644 --- a/docs/content/TopNMetricSpec.md +++ b/docs/content/TopNMetricSpec.md @@ -11,7 +11,7 @@ The topN metric spec specifies how topN values should be sorted. The simplest metric specification is a String value indicating the metric to sort topN results by. They are included in a topN query with: ```json -"metric": +"metric": "" ``` The metric field can also be given as a JSON object. The grammar for dimension values sorted by numeric value is shown below: @@ -19,7 +19,7 @@ The metric field can also be given as a JSON object. The grammar for dimension v ```json "metric": { "type": "numeric", - "metric": "" + "metric": "" } ``` diff --git a/docs/content/TopNQuery.md b/docs/content/TopNQuery.md index 1e537ba9c1d..34fb6bb9335 100644 --- a/docs/content/TopNQuery.md +++ b/docs/content/TopNQuery.md @@ -72,9 +72,9 @@ There are 10 parts to a topN query, but 7 of them are shared with [TimeseriesQue |property|description|required?| |--------|-----------|---------| -|dimension|A JSON object defining the dimension that you want the top taken for. For more info, see [DimensionSpecs](DimensionSpecs.html)|yes| +|dimension|A String or JSON object defining the dimension that you want the top taken for. For more info, see [DimensionSpecs](DimensionSpecs.html)|yes| |threshold|An integer defining the N in the topN (i.e. how many you want in the top list)|yes| -|metric|A JSON object specifying the metric to sort by for the top list. For more info, see [TopNMetricSpec](TopNMetricSpec.html).|yes| +|metric|A String or JSON object specifying the metric to sort by for the top list. For more info, see [TopNMetricSpec](TopNMetricSpec.html).|yes| Please note the context JSON object is also available for topN queries and should be used with the same caution as the timeseries case. The format of the results would look like so: diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index fadc7fbc963..39d908dbbd0 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -75,9 +75,13 @@ Setting up Zookeeper Before we get started, we need to start Apache Zookeeper. ```bash -curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz -tar xzf zookeeper-3.4.5.tar.gz -cd zookeeper-3.4.5 +Download zookeeper from [http://www.apache.org/dyn/closer.cgi/zookeeper/](http://www.apache.org/dyn/closer.cgi/zookeeper/) +Install zookeeper. + +e.g. +curl http://www.gtlib.gatech.edu/pub/apache/zookeeper/zookeeper-3.4.6/zookeeper-3.4.6.tar.gz -o zookeeper-3.4.6.tar.gz +tar xzf zookeeper-3.4.6.tar.gz +cd zookeeper-3.4.6 cp conf/zoo_sample.cfg conf/zoo.cfg ./bin/zkServer.sh start cd .. diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 05a8a44d9f2..0f315f749cc 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -48,9 +48,13 @@ CREATE database druid; #### Setting up Zookeeper ```bash -curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz -tar xzf zookeeper-3.4.5.tar.gz -cd zookeeper-3.4.5 +Download zookeeper from [http://www.apache.org/dyn/closer.cgi/zookeeper/](http://www.apache.org/dyn/closer.cgi/zookeeper/) +Install zookeeper. + +e.g. +curl http://www.gtlib.gatech.edu/pub/apache/zookeeper/zookeeper-3.4.6/zookeeper-3.4.6.tar.gz -o zookeeper-3.4.6.tar.gz +tar xzf zookeeper-3.4.6.tar.gz +cd zookeeper-3.4.6 cp conf/zoo_sample.cfg conf/zoo.cfg ./bin/zkServer.sh start cd .. diff --git a/examples/pom.xml b/examples/pom.xml index 896b772738e..479e8b87272 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -18,106 +18,119 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 - io.druid.extensions - druid-examples - druid-examples - druid-examples + + 4.0.0 + io.druid.extensions + druid-examples + druid-examples + druid-examples - - io.druid - druid - 0.7.0-SNAPSHOT - + + io.druid + druid + 0.7.0-SNAPSHOT + - - - io.druid - druid-server - ${project.parent.version} - - - io.druid - druid-common - ${project.parent.version} - + + + io.druid + druid-server + ${project.parent.version} + + + io.druid + druid-common + ${project.parent.version} + - - org.twitter4j - twitter4j-core - 3.0.3 - - - org.twitter4j - twitter4j-async - 3.0.3 - - - org.twitter4j - twitter4j-stream - 3.0.3 - - - commons-validator - commons-validator - 1.4.0 - + + org.twitter4j + twitter4j-core + 3.0.3 + + + org.twitter4j + twitter4j-async + 3.0.3 + + + org.twitter4j + twitter4j-stream + 3.0.3 + + + commons-validator + commons-validator + 1.4.0 + - - - junit - junit - test - - + + + junit + junit + test + + - - - - maven-shade-plugin - - - package - - shade - - - - ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - - - maven-jar-plugin - - - - test-jar - - - - - - - true - true - - - - - - + + + + maven-shade-plugin + + + package + + shade + + + + ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + test-jar + + + + + + + true + true + + + + + + diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index ac2c29b7405..6b705799afc 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -18,73 +18,86 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 - io.druid.extensions - druid-hdfs-storage - druid-hdfs-storage - druid-hdfs-storage + + 4.0.0 + io.druid.extensions + druid-hdfs-storage + druid-hdfs-storage + druid-hdfs-storage - - io.druid - druid - 0.7.0-SNAPSHOT - + + io.druid + druid + 0.7.0-SNAPSHOT + - - - io.druid - druid-api - - - - net.java.dev.jets3t - jets3t - - - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - org.apache.hadoop - hadoop-client - compile - - - com.metamx - emitter - - - commons-io - commons-io - + + + io.druid + druid-api + + + + net.java.dev.jets3t + jets3t + + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + org.apache.hadoop + hadoop-client + compile + + + com.metamx + emitter + + + commons-io + commons-io + - - - junit - junit - test - - + + + junit + junit + test + + - - - - maven-jar-plugin - - - - true - true - - - - - - + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + true + true + + + + + + diff --git a/histogram/pom.xml b/histogram/pom.xml index a520de6725b..9827e32de49 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -17,54 +17,67 @@ ~ along with this program; if not, write to the Free Software ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 - io.druid.extensions - druid-histogram - druid-histogram - druid-histogram + + 4.0.0 + io.druid.extensions + druid-histogram + druid-histogram + druid-histogram - - io.druid - druid - 0.7.0-SNAPSHOT - + + io.druid + druid + 0.7.0-SNAPSHOT + - - - io.druid - druid-processing - ${project.parent.version} - + + + io.druid + druid-processing + ${project.parent.version} + - - - io.druid - druid-processing - ${project.parent.version} - test - test-jar - - - junit - junit - test - - + + + io.druid + druid-processing + ${project.parent.version} + test + test-jar + + + junit + junit + test + + - - - - maven-jar-plugin - - - - true - true - - - - - - + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + true + true + + + + + + diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index f23382b6de9..da2c6d9ef60 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -18,116 +18,129 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 + + 4.0.0 + io.druid + druid-indexing-hadoop + druid-indexing-hadoop + Druid Indexing Hadoop + + io.druid - druid-indexing-hadoop - druid-indexing-hadoop - Druid Indexing Hadoop + druid + 0.7.0-SNAPSHOT + - - io.druid - druid - 0.7.0-SNAPSHOT - + + + io.druid + druid-server + ${project.parent.version} + - - - io.druid - druid-server - ${project.parent.version} - + + com.metamx + java-util + - - com.metamx - java-util - - - - commons-io - commons-io - - - com.google.guava - guava - - - - net.java.dev.jets3t - jets3t - - - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - org.apache.hadoop - hadoop-client - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - org.jdbi - jdbi - - - joda-time - joda-time - - - com.google.code.findbugs - jsr305 - + + commons-io + commons-io + + + com.google.guava + guava + + + + net.java.dev.jets3t + jets3t + + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + org.apache.hadoop + hadoop-client + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + org.jdbi + jdbi + + + joda-time + joda-time + + + com.google.code.findbugs + jsr305 + - - - junit - junit - test - - + + + junit + junit + test + + - - - - maven-jar-plugin - - - - true - true - - - - - - maven-shade-plugin - - - package - - shade - - - - ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar - - - - - - - + + + + maven-jar-plugin + + + + true + true + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-shade-plugin + + + package + + shade + + + + ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar + + + + + + + diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java index bcab46bcd98..32cd0f4d3f1 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java @@ -38,14 +38,18 @@ public class HadoopDruidIndexerJob implements Jobby @Inject public HadoopDruidIndexerJob( - HadoopDruidIndexerConfig config + HadoopDruidIndexerConfig config, + MetadataStorageUpdaterJobHandler handler ) { config.verify(); this.config = config; if (config.isUpdaterJobSpecSet()) { - metadataStorageUpdaterJob = new MetadataStorageUpdaterJob(config); + metadataStorageUpdaterJob = new MetadataStorageUpdaterJob( + config, + handler + ); } else { metadataStorageUpdaterJob = null; } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/MetadataStorageUpdaterJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/MetadataStorageUpdaterJob.java index 67823462849..21e42f7c145 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/MetadataStorageUpdaterJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/MetadataStorageUpdaterJob.java @@ -19,7 +19,6 @@ package io.druid.indexer; -import com.google.inject.Inject; import io.druid.timeline.DataSegment; import java.util.List; @@ -29,15 +28,15 @@ import java.util.List; public class MetadataStorageUpdaterJob implements Jobby { private final HadoopDruidIndexerConfig config; - - @Inject - private MetadataStorageUpdaterJobHandler handler; + private final MetadataStorageUpdaterJobHandler handler; public MetadataStorageUpdaterJob( - HadoopDruidIndexerConfig config + HadoopDruidIndexerConfig config, + MetadataStorageUpdaterJobHandler handler ) { this.config = config; + this.handler = handler; } @Override @@ -49,4 +48,4 @@ public class MetadataStorageUpdaterJob implements Jobby return true; } -} \ No newline at end of file +} diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/PartitionsSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/PartitionsSpec.java index a36555f0ea8..532e3faba45 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/PartitionsSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/PartitionsSpec.java @@ -26,7 +26,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.Jobby; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = SingleDimensionPartitionsSpec.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = HashedPartitionsSpec.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "dimension", value = SingleDimensionPartitionsSpec.class), @JsonSubTypes.Type(name = "random", value = RandomPartitionsSpec.class), diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java index 4d58693b22c..e6f13a8114c 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java @@ -23,10 +23,16 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Supplier; import io.druid.metadata.MetadataStorageConnectorConfig; +import javax.validation.constraints.NotNull; + /** */ public class MetadataStorageUpdaterJobSpec implements Supplier { + @JsonProperty("type") + @NotNull + public String type; + @JsonProperty("connectURI") public String connectURI; @@ -44,6 +50,11 @@ public class MetadataStorageUpdaterJobSpec implements Supplier - - 4.0.0 + + 4.0.0 + io.druid + druid-indexing-service + druid-indexing-service + druid-indexing-service + + io.druid - druid-indexing-service - druid-indexing-service - druid-indexing-service + druid + 0.7.0-SNAPSHOT + - - io.druid - druid - 0.7.0-SNAPSHOT - + + + io.druid + druid-common + ${project.parent.version} + + + io.druid + druid-server + ${project.parent.version} + + + io.druid + druid-indexing-hadoop + ${project.parent.version} + - - - io.druid - druid-common - ${project.parent.version} - - - io.druid - druid-server - ${project.parent.version} - - - io.druid - druid-indexing-hadoop - ${project.parent.version} - + + + junit + junit + test + + + org.easymock + easymock + test + + + org.apache.curator + curator-test + test + + + org.apache.hadoop + hadoop-client + test + + - - - junit - junit - test - - - org.easymock - easymock - test - - - org.apache.curator - curator-test - test - - - org.apache.hadoop - hadoop-client - test - - - - - - - maven-jar-plugin - - - - true - true - - - - - - + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + true + true + + + + + + diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java index 915fa5a004f..40ab0936488 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java @@ -26,7 +26,8 @@ import com.google.common.base.Throwables; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.http.client.HttpClient; -import com.metamx.http.client.response.ToStringResponseHandler; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; import io.druid.client.selector.Server; import io.druid.curator.discovery.ServerDiscoverySelector; import io.druid.indexing.common.RetryPolicy; @@ -37,6 +38,7 @@ import org.joda.time.Duration; import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; import java.util.Map; public class RemoteTaskActionClient implements TaskActionClient @@ -75,22 +77,25 @@ public class RemoteTaskActionClient implements TaskActionClient while (true) { try { + final Server server; final URI serviceUri; try { - serviceUri = getServiceUri(); + server = getServiceInstance(); + serviceUri = makeServiceUri(server); } catch (Exception e) { + // Want to retry, so throw an IOException. throw new IOException("Failed to locate service uri", e); } - final String response; + final StatusResponseHolder response; log.info("Submitting action for task[%s] to overlord[%s]: %s", task.getId(), serviceUri, taskAction); try { response = httpClient.post(serviceUri.toURL()) .setContent("application/json", dataToSend) - .go(new ToStringResponseHandler(Charsets.UTF_8)) + .go(new StatusResponseHandler(Charsets.UTF_8)) .get(); } catch (Exception e) { @@ -99,13 +104,24 @@ public class RemoteTaskActionClient implements TaskActionClient throw Throwables.propagate(e); } - final Map responseDict = jsonMapper.readValue( - response, new TypeReference>() - { + if (response.getStatus().getCode() / 200 == 1) { + final Map responseDict = jsonMapper.readValue( + response.getContent(), + new TypeReference>() + { + } + ); + return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference()); + } else { + // Want to retry, so throw an IOException. + throw new IOException( + String.format( + "Scary HTTP status returned: %s. Check your overlord[%s] logs for exceptions.", + response.getStatus(), + server.getHost() + ) + ); } - ); - - return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference()); } catch (IOException | ChannelException e) { log.warn(e, "Exception submitting action for task[%s]", task.getId()); @@ -127,13 +143,18 @@ public class RemoteTaskActionClient implements TaskActionClient } } - private URI getServiceUri() throws Exception + private URI makeServiceUri(final Server instance) throws URISyntaxException + { + return new URI(String.format("%s://%s%s", instance.getScheme(), instance.getHost(), "/druid/indexer/v1/action")); + } + + private Server getServiceInstance() { final Server instance = selector.pick(); if (instance == null) { throw new ISE("Cannot find instance of indexer to talk to!"); + } else { + return instance; } - - return new URI(String.format("%s://%s%s", instance.getScheme(), instance.getHost(), "/druid/indexer/v1/action")); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java index 5b6bd37db16..15f89e8cfbd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java @@ -27,7 +27,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMaker; +import io.druid.segment.IndexMerger; import io.druid.segment.IndexableAdapter; import io.druid.segment.QueryableIndexIndexableAdapter; import io.druid.segment.Rowboat; @@ -37,7 +37,6 @@ import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.File; import java.util.List; import java.util.Map; @@ -106,7 +105,7 @@ public class AppendTask extends MergeTaskBase ); } - return IndexMaker.append(adapters, outDir); + return IndexMerger.append(adapters, outDir); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index a0ebfa98dce..da16b81775b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -29,6 +29,7 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import com.google.inject.Injector; import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; import io.druid.guice.ExtensionsConfig; @@ -38,6 +39,7 @@ import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.HadoopDruidIndexerJob; import io.druid.indexer.HadoopIngestionSpec; import io.druid.indexer.Jobby; +import io.druid.indexer.MetadataStorageUpdaterJobHandler; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -63,8 +65,9 @@ public class HadoopIndexTask extends AbstractTask private static final Logger log = new Logger(HadoopIndexTask.class); private static final ExtensionsConfig extensionsConfig; + final static Injector injector = GuiceInjectors.makeStartupInjector(); static { - extensionsConfig = GuiceInjectors.makeStartupInjector().getInstance(ExtensionsConfig.class); + extensionsConfig = injector.getInstance(ExtensionsConfig.class); } private static String getTheDataSource(HadoopIngestionSpec spec, HadoopIngestionSpec config) @@ -184,7 +187,7 @@ public class HadoopIndexTask extends AbstractTask final List extensionURLs = Lists.newArrayList(); for (String coordinate : extensionsConfig.getCoordinates()) { final ClassLoader coordinateLoader = Initialization.getClassLoaderForCoordinates( - aetherClient, coordinate + aetherClient, coordinate, extensionsConfig.getDefaultVersion() ); extensionURLs.addAll(Arrays.asList(((URLClassLoader) coordinateLoader).getURLs())); } @@ -197,7 +200,7 @@ public class HadoopIndexTask extends AbstractTask // put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts for (String hadoopDependencyCoordinate : finalHadoopDependencyCoordinates) { final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates( - aetherClient, hadoopDependencyCoordinate + aetherClient, hadoopDependencyCoordinate, extensionsConfig.getDefaultVersion() ); driverURLs.addAll(Arrays.asList(((URLClassLoader) hadoopLoader).getURLs())); } @@ -288,7 +291,10 @@ public class HadoopIndexTask extends AbstractTask .withTuningConfig(theSchema.getTuningConfig().withVersion(version)) ); - HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(config); + HadoopDruidIndexerJob job = new HadoopDruidIndexerJob( + config, + injector.getInstance(MetadataStorageUpdaterJobHandler.class) + ); log.info("Starting a hadoop index generator job..."); if (job.run()) { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java index d856d1505bc..9bca4a3eee5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java @@ -28,7 +28,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMaker; +import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.timeline.DataSegment; @@ -60,7 +60,7 @@ public class MergeTask extends MergeTaskBase public File merge(final Map segments, final File outDir) throws Exception { - return IndexMaker.mergeQueryableIndex( + return IndexMerger.mergeQueryableIndex( Lists.transform( ImmutableList.copyOf(segments.values()), new Function() diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index e0c5fb43ebe..d501335c9fe 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -18,61 +18,74 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 - io.druid.extensions - druid-kafka-eight - druid-kafka-eight - druid-kafka-eight + + 4.0.0 + io.druid.extensions + druid-kafka-eight + druid-kafka-eight + druid-kafka-eight - - io.druid - druid - 0.7.0-SNAPSHOT - + + io.druid + druid + 0.7.0-SNAPSHOT + - - - io.druid - druid-api - - - org.apache.kafka - kafka_2.9.2 - 0.8.0 - - - log4j - log4j - - - org.apache.zookeeper - zookeeper - - - + + + io.druid + druid-api + + + org.apache.kafka + kafka_2.9.2 + 0.8.0 + + + log4j + log4j + + + org.apache.zookeeper + zookeeper + + + - - - junit - junit - test - - + + + junit + junit + test + + - - - - maven-jar-plugin - - - - true - true - - - - - - + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + true + true + + + + + + diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index a5e6ef4b806..ced7ee5192b 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -18,60 +18,73 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 - io.druid.extensions - druid-kafka-seven - druid-kafka-seven - druid-kafka-seven + + 4.0.0 + io.druid.extensions + druid-kafka-seven + druid-kafka-seven + druid-kafka-seven - - io.druid - druid - 0.7.0-SNAPSHOT - + + io.druid + druid + 0.7.0-SNAPSHOT + - - - io.druid - druid-api - - - kafka - core-kafka - 0.7.2-mmx4 - - - log4j - log4j - - - org.jboss.netty - netty - - - + + + io.druid + druid-api + + + kafka + core-kafka + 0.7.2-mmx4 + + + log4j + log4j + + + org.jboss.netty + netty + + + - - - junit - junit - test - - - - - - maven-jar-plugin - - - - true - true - - - - - - + + + junit + junit + test + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + true + true + + + + + + diff --git a/mysql-metadata-storage/pom.xml b/mysql-metadata-storage/pom.xml index 8cc1ca48c45..2fa9c9ffda0 100644 --- a/mysql-metadata-storage/pom.xml +++ b/mysql-metadata-storage/pom.xml @@ -61,6 +61,18 @@ + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + maven-jar-plugin diff --git a/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnector.java b/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnector.java index c27ace5a6cf..39b2b6ed0c8 100644 --- a/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnector.java +++ b/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnector.java @@ -72,7 +72,7 @@ public class MySQLConnector extends SQLMetadataConnector } @Override - protected boolean tableExists(Handle handle, String tableName) + public boolean tableExists(Handle handle, String tableName) { return !handle.createQuery("SHOW tables LIKE :tableName") .bind("tableName", tableName) diff --git a/pom.xml b/pom.xml index dba0c8c2207..13d7ba5c51f 100644 --- a/pom.xml +++ b/pom.xml @@ -92,7 +92,7 @@ com.metamx bytebuffer-collections - 0.0.4 + 0.1.1 com.metamx @@ -194,11 +194,6 @@ curator-x-discovery ${apache.curator.version} - - it.uniroma3.mat - extendedset - 1.3.7 - com.google.guava guava @@ -447,6 +442,16 @@ mapdb 1.0.6 + + org.apache.derby + derbynet + 10.11.1.1 + + + org.apache.derby + derbyclient + 10.11.1.1 + @@ -468,6 +473,12 @@ 4.11 test + + com.carrotsearch + junit-benchmarks + 0.7.2 + test + com.google.caliper caliper diff --git a/postgresql-metadata-storage/pom.xml b/postgresql-metadata-storage/pom.xml index e0bffaccdae..f88f4515e19 100644 --- a/postgresql-metadata-storage/pom.xml +++ b/postgresql-metadata-storage/pom.xml @@ -61,6 +61,18 @@ + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + maven-jar-plugin diff --git a/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLConnector.java b/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLConnector.java index 13dbbab324a..b2114da2093 100644 --- a/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLConnector.java +++ b/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLConnector.java @@ -65,7 +65,7 @@ public class PostgreSQLConnector extends SQLMetadataConnector } @Override - protected boolean tableExists(final Handle handle, final String tableName) + public boolean tableExists(final Handle handle, final String tableName) { return !handle.createQuery( "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = 'public' AND tablename LIKE :tableName" diff --git a/processing/pom.xml b/processing/pom.xml index 2f60f53d2bc..1d74887c994 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -20,116 +20,130 @@ - 4.0.0 + 4.0.0 + io.druid + druid-processing + druid-processing + A module that is everything required to understands Druid Segments + + io.druid - druid-processing - druid-processing - A module that is everything required to understands Druid Segments + druid + 0.7.0-SNAPSHOT + - - io.druid - druid - 0.7.0-SNAPSHOT - - - - - io.druid - druid-common - ${project.parent.version} - - - com.metamx - bytebuffer-collections - - - com.metamx - emitter - - - com.ning - compress-lzf - - - org.skife.config - config-magic - - - it.uniroma3.mat - extendedset - - - org.slf4j - slf4j-log4j12 - - - com.google.protobuf - protobuf-java - - - commons-io - commons-io - - - com.ibm.icu - icu4j - - - org.mozilla - rhino - 1.7R4 - - - com.davekoelle - alphanum - - - net.jpountz.lz4 - lz4 - - - org.mapdb - mapdb - + + + io.druid + druid-common + ${project.parent.version} + + + com.metamx + bytebuffer-collections + + + com.metamx + emitter + + + com.ning + compress-lzf + + + org.skife.config + config-magic + + + org.slf4j + slf4j-log4j12 + + + com.google.protobuf + protobuf-java + + + commons-io + commons-io + + + com.ibm.icu + icu4j + + + org.mozilla + rhino + 1.7R4 + + + com.davekoelle + alphanum + + + net.jpountz.lz4 + lz4 + + + org.mapdb + mapdb + - - - junit - junit - test - - - org.easymock - easymock - test - - - com.google.caliper - caliper - - + + + junit + junit + test + + + com.carrotsearch + junit-benchmarks + test + + + org.easymock + easymock + test + + + com.google.caliper + caliper + test + + - - - - maven-jar-plugin - - - - test-jar - - - - - - - true - true - - - - - - + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + test-jar + + + + + + + true + true + + + + + + diff --git a/processing/src/main/java/io/druid/granularity/PeriodGranularity.java b/processing/src/main/java/io/druid/granularity/PeriodGranularity.java index 2aef4ff4044..c3e4feff90d 100644 --- a/processing/src/main/java/io/druid/granularity/PeriodGranularity.java +++ b/processing/src/main/java/io/druid/granularity/PeriodGranularity.java @@ -326,7 +326,7 @@ public class PeriodGranularity extends BaseQueryGranularity @Override public byte[] cacheKey() { - return (period.toString() + ":" + chronology.getZone().toString()).getBytes(Charsets.UTF_8); + return (period.toString() + ":" + chronology.getZone().toString() + ":" + origin).getBytes(Charsets.UTF_8); } @Override diff --git a/processing/src/main/java/io/druid/guice/ExtensionsConfig.java b/processing/src/main/java/io/druid/guice/ExtensionsConfig.java index b904b421045..f8fc723f902 100644 --- a/processing/src/main/java/io/druid/guice/ExtensionsConfig.java +++ b/processing/src/main/java/io/druid/guice/ExtensionsConfig.java @@ -29,6 +29,8 @@ import java.util.List; */ public class ExtensionsConfig { + public static final String PACKAGE_VERSION = ExtensionsConfig.class.getPackage().getImplementationVersion(); + @JsonProperty @NotNull private boolean searchCurrentClassloader = true; @@ -37,6 +39,10 @@ public class ExtensionsConfig @NotNull private List coordinates = ImmutableList.of(); + // default version to use for extensions without version info + @JsonProperty + private String defaultVersion; + @JsonProperty @NotNull private String localRepository = String.format("%s/%s", System.getProperty("user.home"), ".m2/repository"); @@ -58,6 +64,11 @@ public class ExtensionsConfig return coordinates; } + public String getDefaultVersion() + { + return defaultVersion != null ? defaultVersion : PACKAGE_VERSION; + } + public String getLocalRepository() { return localRepository; @@ -74,6 +85,7 @@ public class ExtensionsConfig return "ExtensionsConfig{" + "searchCurrentClassloader=" + searchCurrentClassloader + ", coordinates=" + coordinates + + ", defaultVersion='" + getDefaultVersion() + '\'' + ", localRepository='" + localRepository + '\'' + ", remoteRepositories=" + remoteRepositories + '}'; diff --git a/processing/src/main/java/io/druid/jackson/JacksonModule.java b/processing/src/main/java/io/druid/jackson/JacksonModule.java index 16d19cf9544..94e4e081deb 100644 --- a/processing/src/main/java/io/druid/jackson/JacksonModule.java +++ b/processing/src/main/java/io/druid/jackson/JacksonModule.java @@ -48,7 +48,9 @@ public class JacksonModule implements Module @Provides @LazySingleton @Smile public ObjectMapper smileMapper() { - ObjectMapper retVal = new DefaultObjectMapper(new SmileFactory()); + final SmileFactory smileFactory = new SmileFactory(); + smileFactory.delegateToTextual(true); + final ObjectMapper retVal = new DefaultObjectMapper(smileFactory); retVal.getFactory().setCodec(retVal); return retVal; } diff --git a/processing/src/main/java/io/druid/query/BySegmentResultValue.java b/processing/src/main/java/io/druid/query/BySegmentResultValue.java index 3d2dc0802f3..fdc27d9c652 100644 --- a/processing/src/main/java/io/druid/query/BySegmentResultValue.java +++ b/processing/src/main/java/io/druid/query/BySegmentResultValue.java @@ -19,15 +19,17 @@ package io.druid.query; +import org.joda.time.Interval; + import java.util.List; /** */ public interface BySegmentResultValue { - public List> getResults(); + public List getResults(); public String getSegmentId(); - public String getIntervalString(); + public Interval getInterval(); } diff --git a/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java b/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java index 3a3544cf47a..0d779733e66 100644 --- a/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java +++ b/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java @@ -26,7 +26,7 @@ import java.util.List; /** */ -public class BySegmentResultValueClass +public class BySegmentResultValueClass implements BySegmentResultValue { private final List results; private final String segmentId; @@ -43,18 +43,21 @@ public class BySegmentResultValueClass this.interval = interval; } + @Override @JsonProperty("results") public List getResults() { return results; } + @Override @JsonProperty("segment") public String getSegmentId() { return segmentId; } + @Override @JsonProperty("interval") public Interval getInterval() { diff --git a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java index 10bff0a65e2..1794024b60c 100644 --- a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java +++ b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java @@ -84,7 +84,7 @@ public class FinalizeResultsQueryRunner implements QueryRunner throw new ISE("Cannot have a null result!"); } - BySegmentResultValueClass resultsClass = result.getValue(); + BySegmentResultValue resultsClass = result.getValue(); return (T) new Result( result.getTimestamp(), diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 9f3921bd87d..9f6fd5d474a 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -22,11 +22,12 @@ package io.druid.query; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; -import com.google.common.collect.MapMaker; import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; import com.metamx.common.guava.YieldingSequenceBase; +import com.metamx.emitter.EmittingLogger; import io.druid.query.spec.MultipleSpecificSegmentSpec; import io.druid.segment.SegmentMissingException; @@ -36,18 +37,22 @@ import java.util.Map; public class RetryQueryRunner implements QueryRunner { public static String MISSING_SEGMENTS_KEY = "missingSegments"; + private static final EmittingLogger log = new EmittingLogger(RetryQueryRunner.class); private final QueryRunner baseRunner; + private final QueryToolChest> toolChest; private final RetryQueryRunnerConfig config; private final ObjectMapper jsonMapper; public RetryQueryRunner( QueryRunner baseRunner, + QueryToolChest> toolChest, RetryQueryRunnerConfig config, ObjectMapper jsonMapper ) { this.baseRunner = baseRunner; + this.toolChest = toolChest; this.config = config; this.jsonMapper = jsonMapper; } @@ -55,7 +60,8 @@ public class RetryQueryRunner implements QueryRunner @Override public Sequence run(final Query query, final Map context) { - final Sequence returningSeq = baseRunner.run(query, context); + final List> listOfSequences = Lists.newArrayList(); + listOfSequences.add(baseRunner.run(query, context)); return new YieldingSequenceBase() { @@ -64,33 +70,32 @@ public class RetryQueryRunner implements QueryRunner OutType initValue, YieldingAccumulator accumulator ) { - Yielder yielder = returningSeq.toYielder(initValue, accumulator); - final List missingSegments = getMissingSegments(context); - if (missingSegments.isEmpty()) { - return yielder; - } + if (!missingSegments.isEmpty()) { + for (int i = 0; i < config.getNumTries(); i++) { + log.info("[%,d] missing segments found. Retry attempt [%,d]", missingSegments.size(), i); - for (int i = 0; i < config.numTries(); i++) { - context.put(MISSING_SEGMENTS_KEY, Lists.newArrayList()); - final Query retryQuery = query.withQuerySegmentSpec( - new MultipleSpecificSegmentSpec( - missingSegments - ) - ); - yielder = baseRunner.run(retryQuery, context).toYielder(initValue, accumulator); - if (getMissingSegments(context).isEmpty()) { - break; + context.put(MISSING_SEGMENTS_KEY, Lists.newArrayList()); + final Query retryQuery = query.withQuerySegmentSpec( + new MultipleSpecificSegmentSpec( + missingSegments + ) + ); + Sequence retrySequence = baseRunner.run(retryQuery, context); + listOfSequences.add(retrySequence); + if (getMissingSegments(context).isEmpty()) { + break; + } + } + + final List finalMissingSegs = getMissingSegments(context); + if (!config.isReturnPartialResults() && !finalMissingSegs.isEmpty()) { + throw new SegmentMissingException("No results found for segments[%s]", finalMissingSegs); } } - final List finalMissingSegs = getMissingSegments(context); - if (!config.returnPartialResults() && !finalMissingSegs.isEmpty()) { - throw new SegmentMissingException("No results found for segments[%s]", finalMissingSegs); - } - - return yielder; + return toolChest.mergeSequencesUnordered(Sequences.simple(listOfSequences)).toYielder(initValue, accumulator); } }; } diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java index 2b8bb730b68..e75d5af09d2 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java @@ -24,10 +24,16 @@ import com.fasterxml.jackson.annotation.JsonProperty; public class RetryQueryRunnerConfig { @JsonProperty - private int numTries = 0; - @JsonProperty - private boolean returnPartialResults = false; + private int numTries = 1; - public int numTries() { return numTries; } - public boolean returnPartialResults() { return returnPartialResults; } + public int getNumTries() + { + return numTries; + } + + // exists for testing and overrides + public boolean isReturnPartialResults() + { + return false; + } } diff --git a/processing/src/main/java/io/druid/query/filter/BitmapIndexSelector.java b/processing/src/main/java/io/druid/query/filter/BitmapIndexSelector.java index c0a2abca40b..147d4b6b720 100644 --- a/processing/src/main/java/io/druid/query/filter/BitmapIndexSelector.java +++ b/processing/src/main/java/io/druid/query/filter/BitmapIndexSelector.java @@ -19,9 +19,10 @@ package io.druid.query.filter; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.spatial.ImmutableRTree; import io.druid.segment.data.Indexed; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** */ @@ -29,7 +30,8 @@ public interface BitmapIndexSelector { public Indexed getDimensionValues(String dimension); public int getNumRows(); - public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value); - public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx); + public BitmapFactory getBitmapFactory(); + public ImmutableBitmap getBitmapIndex(String dimension, String value); + public ImmutableBitmap getBitmapIndex(String dimension, int idx); public ImmutableRTree getSpatialIndex(String dimension); } diff --git a/processing/src/main/java/io/druid/query/filter/Filter.java b/processing/src/main/java/io/druid/query/filter/Filter.java index 73d39074c75..379ae7a2014 100644 --- a/processing/src/main/java/io/druid/query/filter/Filter.java +++ b/processing/src/main/java/io/druid/query/filter/Filter.java @@ -19,12 +19,12 @@ package io.druid.query.filter; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; +import com.metamx.collections.bitmap.ImmutableBitmap; /** */ public interface Filter { - public ImmutableConciseSet goConcise(BitmapIndexSelector selector); + public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector); public ValueMatcher makeMatcher(ValueMatcherFactory factory); } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java index 02e3a0c16d2..8f51d95a8f9 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java @@ -122,7 +122,7 @@ public class SegmentAnalyzer String value = bitmapIndex.getValue(i); if (value != null) { - size += value.getBytes(Charsets.UTF_8).length * bitmapIndex.getConciseSet(value).size(); + size += value.getBytes(Charsets.UTF_8).length * bitmapIndex.getBitmap(value).size(); } } diff --git a/processing/src/main/java/io/druid/query/search/BySegmentSearchResultValue.java b/processing/src/main/java/io/druid/query/search/BySegmentSearchResultValue.java index 0ecbdf509c2..c77b25f19bd 100644 --- a/processing/src/main/java/io/druid/query/search/BySegmentSearchResultValue.java +++ b/processing/src/main/java/io/druid/query/search/BySegmentSearchResultValue.java @@ -24,28 +24,30 @@ import com.fasterxml.jackson.annotation.JsonValue; import io.druid.query.BySegmentResultValue; import io.druid.query.Result; import io.druid.query.search.search.SearchHit; +import org.joda.time.Interval; import java.util.List; /** */ -public class BySegmentSearchResultValue extends SearchResultValue implements BySegmentResultValue +public class BySegmentSearchResultValue extends SearchResultValue + implements BySegmentResultValue> { private final List> results; private final String segmentId; - private final String intervalString; + private final Interval interval; public BySegmentSearchResultValue( @JsonProperty("results") List> results, @JsonProperty("segment") String segmentId, - @JsonProperty("interval") String intervalString + @JsonProperty("interval") Interval interval ) { super(null); this.results = results; this.segmentId = segmentId; - this.intervalString = intervalString; + this.interval = interval; } @Override @@ -71,9 +73,9 @@ public class BySegmentSearchResultValue extends SearchResultValue implements ByS @Override @JsonProperty("interval") - public String getIntervalString() + public Interval getInterval() { - return intervalString; + return interval; } @Override @@ -82,7 +84,7 @@ public class BySegmentSearchResultValue extends SearchResultValue implements ByS return "BySegmentSearchResultValue{" + "results=" + results + ", segmentId='" + segmentId + '\'' + - ", intervalString='" + intervalString + '\'' + + ", interval='" + interval.toString() + '\'' + '}'; } } diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index d6bcd61cfe1..79d9686cee8 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -329,7 +329,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest> dimsToSearch = dimensions; } + BitmapFactory bitmapFactory = index.getBitmapFactoryForDimensions(); - final ImmutableConciseSet baseFilter; + final ImmutableBitmap baseFilter; if (filter == null) { - baseFilter = ImmutableConciseSet.complement(new ImmutableConciseSet(), index.getNumRows()); + baseFilter = bitmapFactory.complement(bitmapFactory.makeEmptyImmutableBitmap(), index.getNumRows()); } else { - baseFilter = filter.goConcise(new ColumnSelectorBitmapIndexSelector(index)); + ColumnSelectorBitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(bitmapFactory, index); + baseFilter = filter.getBitmapIndex(selector); } for (String dimension : dimsToSearch) { @@ -113,7 +118,7 @@ public class SearchQueryRunner implements QueryRunner> for (int i = 0; i < bitmapIndex.getCardinality(); ++i) { String dimVal = Strings.nullToEmpty(bitmapIndex.getValue(i)); if (searchQuerySpec.accept(dimVal) && - ImmutableConciseSet.intersection(baseFilter, bitmapIndex.getConciseSet(i)).size() > 0) { + bitmapFactory.intersection(Arrays.asList(baseFilter, bitmapIndex.getBitmap(i))).size() > 0) { retVal.add(new SearchHit(dimension, dimVal)); if (retVal.size() >= limit) { return makeReturnResult(limit, retVal); diff --git a/processing/src/main/java/io/druid/query/search/search/LexicographicSearchSortSpec.java b/processing/src/main/java/io/druid/query/search/search/LexicographicSearchSortSpec.java index 9f84a47292f..04c931112f1 100644 --- a/processing/src/main/java/io/druid/query/search/search/LexicographicSearchSortSpec.java +++ b/processing/src/main/java/io/druid/query/search/search/LexicographicSearchSortSpec.java @@ -41,7 +41,11 @@ public class LexicographicSearchSortSpec implements SearchSortSpec @Override public int compare(SearchHit searchHit, SearchHit searchHit1) { - return searchHit.getValue().compareTo(searchHit1.getValue()); + int retVal = searchHit.getValue().compareTo(searchHit1.getValue()); + if (retVal == 0) { + retVal = searchHit.getDimension().compareTo(searchHit1.getDimension()); + } + return retVal; } }; } diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index b8b25922e26..b8b68f86895 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -23,7 +23,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; -import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; import io.druid.query.Query; @@ -68,20 +67,7 @@ public class SpecificSegmentQueryRunner implements QueryRunner @Override public Sequence call() throws Exception { - Sequence returningSeq; - try { - returningSeq = base.run(query, context); - } - catch (SegmentMissingException e) { - List missingSegments = (List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY); - if (missingSegments == null) { - missingSegments = Lists.newArrayList(); - context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, missingSegments); - } - missingSegments.add(specificSpec.getDescriptor()); - returningSeq = Sequences.empty(); - } - return returningSeq; + return base.run(query, context); } } ); @@ -97,7 +83,18 @@ public class SpecificSegmentQueryRunner implements QueryRunner @Override public OutType call() throws Exception { - return baseSequence.accumulate(initValue, accumulator); + try { + return baseSequence.accumulate(initValue, accumulator); + } + catch (SegmentMissingException e) { + List missingSegments = (List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY); + if (missingSegments == null) { + missingSegments = Lists.newArrayList(); + context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, missingSegments); + } + missingSegments.add(specificSpec.getDescriptor()); + return initValue; + } } } ); diff --git a/processing/src/main/java/io/druid/query/topn/BySegmentTopNResultValue.java b/processing/src/main/java/io/druid/query/topn/BySegmentTopNResultValue.java index 56849d6fd47..009676eb8f9 100644 --- a/processing/src/main/java/io/druid/query/topn/BySegmentTopNResultValue.java +++ b/processing/src/main/java/io/druid/query/topn/BySegmentTopNResultValue.java @@ -23,30 +23,32 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonValue; import io.druid.query.BySegmentResultValue; +import io.druid.query.BySegmentResultValueClass; import io.druid.query.Result; +import org.joda.time.Interval; import java.util.List; /** */ -public class BySegmentTopNResultValue extends TopNResultValue implements BySegmentResultValue +public class BySegmentTopNResultValue extends TopNResultValue implements BySegmentResultValue> { private final List> results; private final String segmentId; - private final String intervalString; + private final Interval interval; @JsonCreator public BySegmentTopNResultValue( @JsonProperty("results") List> results, @JsonProperty("segment") String segmentId, - @JsonProperty("interval") String intervalString + @JsonProperty("interval") Interval interval ) { super(null); this.results = results; this.segmentId = segmentId; - this.intervalString = intervalString; + this.interval = interval; } @Override @@ -73,9 +75,9 @@ public class BySegmentTopNResultValue extends TopNResultValue implements BySegme @Override @JsonProperty("interval") - public String getIntervalString() + public Interval getInterval() { - return intervalString; + return interval; } @Override @@ -84,7 +86,7 @@ public class BySegmentTopNResultValue extends TopNResultValue implements BySegme return "BySegmentTopNResultValue{" + "results=" + results + ", segmentId='" + segmentId + '\'' + - ", intervalString='" + intervalString + '\'' + + ", interval='" + interval.toString() + '\'' + '}'; } } diff --git a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java index caf892f292d..294c13d7db5 100644 --- a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -40,6 +40,7 @@ public class PooledTopNAlgorithm private final Capabilities capabilities; private final TopNQuery query; private final StupidPool bufferPool; + private static final int AGG_UNROLL_COUNT = 8; // Must be able to fit loop below public PooledTopNAlgorithm( Capabilities capabilities, @@ -143,13 +144,30 @@ public class PooledTopNAlgorithm { return makeBufferAggregators(params.getCursor(), query.getAggregatorSpecs()); } - + /** + * Use aggressive loop unrolling to aggregate the data + * + * How this works: The aggregates are evaluated AGG_UNROLL_COUNT at a time. This was chosen to be 8 rather arbitrarily. + * The offsets into the output buffer are precalculated and stored in aggregatorOffsets + * + * For queries whose aggregate count is less than AGG_UNROLL_COUNT, the aggregates evaluted in a switch statement. + * See http://en.wikipedia.org/wiki/Duff's_device for more information on this kind of approach + * + * This allows out of order execution of the code. In local tests, the JVM inlines all the way to this function. + * + * If there are more than AGG_UNROLL_COUNT aggregates, then the remainder is calculated with the switch, and the + * blocks of AGG_UNROLL_COUNT are calculated in a partially unrolled for-loop. + * + * Putting the switch first allows for optimization for the common case (less than AGG_UNROLL_COUNT aggs) but + * still optimizes the high quantity of aggregate queries which benefit greatly from any speed improvements + * (they simply take longer to start with). + */ @Override protected void scanAndAggregate( - PooledTopNParams params, - int[] positions, - BufferAggregator[] theAggregators, - int numProcessed + final PooledTopNParams params, + final int[] positions, + final BufferAggregator[] theAggregators, + final int numProcessed ) { final ByteBuffer resultsBuf = params.getResultsBuf(); @@ -158,35 +176,102 @@ public class PooledTopNAlgorithm final Cursor cursor = params.getCursor(); final DimensionSelector dimSelector = params.getDimSelector(); + final int[] aggregatorOffsets = new int[aggregatorSizes.length]; + for (int j = 0, offset = 0; j < aggregatorSizes.length; ++j) { + aggregatorOffsets[j] = offset; + offset += aggregatorSizes[j]; + } + + final int aggSize = theAggregators.length; + final int aggExtra = aggSize % AGG_UNROLL_COUNT; + while (!cursor.isDone()) { final IndexedInts dimValues = dimSelector.getRow(); - for (int i = 0; i < dimValues.size(); ++i) { - final int dimIndex = dimValues.get(i); - int position = positions[dimIndex]; - switch (position) { - case SKIP_POSITION_VALUE: - break; - case INIT_POSITION_VALUE: - positions[dimIndex] = (dimIndex - numProcessed) * numBytesPerRecord; - position = positions[dimIndex]; - for (int j = 0; j < theAggregators.length; ++j) { - theAggregators[j].init(resultsBuf, position); - position += aggregatorSizes[j]; - } - position = positions[dimIndex]; - default: - for (int j = 0; j < theAggregators.length; ++j) { - theAggregators[j].aggregate(resultsBuf, position); - position += aggregatorSizes[j]; - } - } + final int dimSize = dimValues.size(); + final int dimExtra = dimSize % AGG_UNROLL_COUNT; + switch(dimExtra){ + case 7: + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(6)); + case 6: + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(5)); + case 5: + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(4)); + case 4: + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(3)); + case 3: + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(2)); + case 2: + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(1)); + case 1: + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(0)); + } + for (int i = dimExtra; i < dimSize; i += AGG_UNROLL_COUNT) { + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(i)); + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(i+1)); + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(i+2)); + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(i+3)); + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(i+4)); + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(i+5)); + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(i+6)); + aggregateDimValue(positions, theAggregators, numProcessed, resultsBuf, numBytesPerRecord, aggregatorOffsets, aggSize, aggExtra, dimValues.get(i+7)); } - cursor.advance(); } } + private static void aggregateDimValue( + final int[] positions, + final BufferAggregator[] theAggregators, + final int numProcessed, + final ByteBuffer resultsBuf, + final int numBytesPerRecord, + final int[] aggregatorOffsets, + final int aggSize, + final int aggExtra, + final int dimIndex + ) + { + if (SKIP_POSITION_VALUE == positions[dimIndex]) { + return; + } + if (INIT_POSITION_VALUE == positions[dimIndex]) { + positions[dimIndex] = (dimIndex - numProcessed) * numBytesPerRecord; + final int pos = positions[dimIndex]; + for (int j = 0; j < aggSize; ++j) { + theAggregators[j].init(resultsBuf, pos + aggregatorOffsets[j]); + } + } + final int position = positions[dimIndex]; + + switch(aggExtra) { + case 7: + theAggregators[6].aggregate(resultsBuf, position + aggregatorOffsets[6]); + case 6: + theAggregators[5].aggregate(resultsBuf, position + aggregatorOffsets[5]); + case 5: + theAggregators[4].aggregate(resultsBuf, position + aggregatorOffsets[4]); + case 4: + theAggregators[3].aggregate(resultsBuf, position + aggregatorOffsets[3]); + case 3: + theAggregators[2].aggregate(resultsBuf, position + aggregatorOffsets[2]); + case 2: + theAggregators[1].aggregate(resultsBuf, position + aggregatorOffsets[1]); + case 1: + theAggregators[0].aggregate(resultsBuf, position + aggregatorOffsets[0]); + } + for (int j = aggExtra; j < aggSize; j += AGG_UNROLL_COUNT) { + theAggregators[j].aggregate(resultsBuf, position + aggregatorOffsets[j]); + theAggregators[j+1].aggregate(resultsBuf, position + aggregatorOffsets[j+1]); + theAggregators[j+2].aggregate(resultsBuf, position + aggregatorOffsets[j+2]); + theAggregators[j+3].aggregate(resultsBuf, position + aggregatorOffsets[j+3]); + theAggregators[j+4].aggregate(resultsBuf, position + aggregatorOffsets[j+4]); + theAggregators[j+5].aggregate(resultsBuf, position + aggregatorOffsets[j+5]); + theAggregators[j+6].aggregate(resultsBuf, position + aggregatorOffsets[j+6]); + theAggregators[j+7].aggregate(resultsBuf, position + aggregatorOffsets[j+7]); + } + } + @Override protected void updateResults( PooledTopNParams params, diff --git a/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java index 4a40f4bb2d5..84cf7c83e38 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java @@ -19,8 +19,8 @@ package io.druid.query.topn; -import com.google.common.collect.Maps; -import com.google.common.collect.MinMaxPriorityQueue; +import com.google.common.base.Function; +import com.google.common.collect.Lists; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorUtil; @@ -28,22 +28,50 @@ import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; import org.joda.time.DateTime; -import java.util.ArrayList; +import java.util.Arrays; import java.util.Comparator; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.PriorityQueue; + /** + * */ public class TopNNumericResultBuilder implements TopNResultBuilder { + private final DateTime timestamp; private final DimensionSpec dimSpec; private final String metricName; private final List aggFactories; private final List postAggs; - private MinMaxPriorityQueue pQueue = null; + private final PriorityQueue pQueue; + private final Comparator dimValComparator; + private static final Comparator dimNameComparator = new Comparator() + { + @Override + public int compare(String o1, String o2) + { + int retval; + if (null == o1) { + if (null == o2) { + retval = 0; + } else { + retval = -1; + } + } else if (null == o2) { + retval = 1; + } else { + retval = o1.compareTo(o2); + } + return retval; + } + }; + private final int threshold; + private final Comparator metricComparator; public TopNNumericResultBuilder( DateTime timestamp, @@ -60,18 +88,35 @@ public class TopNNumericResultBuilder implements TopNResultBuilder this.metricName = metricName; this.aggFactories = aggFactories; this.postAggs = AggregatorUtil.pruneDependentPostAgg(postAggs, this.metricName); + this.threshold = threshold; + this.metricComparator = comparator; + this.dimValComparator = new Comparator() + { + @Override + public int compare(DimValHolder d1, DimValHolder d2) + { + int retVal = metricComparator.compare(d1.getTopNMetricVal(), d2.getTopNMetricVal()); - instantiatePQueue(threshold, comparator); + if (retVal == 0) { + retVal = dimNameComparator.compare(d1.getDimName(), d2.getDimName()); + } + + return retVal; + } + }; + + // The logic in addEntry first adds, then removes if needed. So it can at any point have up to threshold + 1 entries. + pQueue = new PriorityQueue<>(this.threshold + 1, this.dimValComparator); } @Override - public TopNResultBuilder addEntry( + public TopNNumericResultBuilder addEntry( String dimName, Object dimValIndex, Object[] metricVals ) { - Map metricValues = Maps.newLinkedHashMap(); + final Map metricValues = new LinkedHashMap<>(metricVals.length + postAggs.size()); metricValues.put(dimSpec.getOutputName(), dimName); @@ -85,27 +130,47 @@ public class TopNNumericResultBuilder implements TopNResultBuilder } Object topNMetricVal = metricValues.get(metricName); - pQueue.add( - new DimValHolder.Builder().withTopNMetricVal(topNMetricVal) - .withDirName(dimName) - .withDimValIndex(dimValIndex) - .withMetricValues(metricValues) - .build() - ); + + if (shouldAdd(topNMetricVal)) { + DimValHolder dimValHolder = new DimValHolder.Builder() + .withTopNMetricVal(topNMetricVal) + .withDirName(dimName) + .withDimValIndex(dimValIndex) + .withMetricValues(metricValues) + .build(); + pQueue.add(dimValHolder); + } + if (this.pQueue.size() > this.threshold) { + pQueue.poll(); + } return this; } + private boolean shouldAdd(Object topNMetricVal) + { + final boolean belowThreshold = pQueue.size() < this.threshold; + final boolean belowMax = belowThreshold + || this.metricComparator.compare(pQueue.peek().getTopNMetricVal(), topNMetricVal) < 0; + return belowMax; + } + @Override public TopNResultBuilder addEntry(DimensionAndMetricValueExtractor dimensionAndMetricValueExtractor) { - pQueue.add( - new DimValHolder.Builder().withTopNMetricVal(dimensionAndMetricValueExtractor.getDimensionValue(metricName)) - .withDirName(dimSpec.getOutputName()) - .withMetricValues(dimensionAndMetricValueExtractor.getBaseObject()) - .build() - ); + final Object dimValue = dimensionAndMetricValueExtractor.getDimensionValue(metricName); + if (shouldAdd(dimValue)) { + final DimValHolder valHolder = new DimValHolder.Builder() + .withTopNMetricVal(dimValue) + .withDirName(dimensionAndMetricValueExtractor.getStringDimensionValue(dimSpec.getOutputName())) + .withMetricValues(dimensionAndMetricValueExtractor.getBaseObject()) + .build(); + pQueue.add(valHolder); + } + if (pQueue.size() > this.threshold) { + pQueue.poll(); // throw away + } return this; } @@ -118,41 +183,41 @@ public class TopNNumericResultBuilder implements TopNResultBuilder @Override public Result build() { - // Pull out top aggregated values - List> values = new ArrayList>(pQueue.size()); - while (!pQueue.isEmpty()) { - values.add(pQueue.remove().getMetricValues()); - } - - return new Result( - timestamp, - new TopNResultValue(values) - ); - } - - private void instantiatePQueue(int threshold, final Comparator comparator) - { - this.pQueue = MinMaxPriorityQueue.orderedBy( - new Comparator() + final DimValHolder[] holderValueArray = pQueue.toArray(new DimValHolder[0]); + Arrays.sort( + holderValueArray, new Comparator() { @Override public int compare(DimValHolder d1, DimValHolder d2) { - int retVal = comparator.compare(d2.getTopNMetricVal(), d1.getTopNMetricVal()); + // Values flipped compared to earlier + int retVal = metricComparator.compare(d2.getTopNMetricVal(), d1.getTopNMetricVal()); if (retVal == 0) { - if (d1.getDimName() == null) { - retVal = -1; - } else if (d2.getDimName() == null) { - retVal = 1; - } else { - retVal = d1.getDimName().compareTo(d2.getDimName()); - } + retVal = dimNameComparator.compare(d1.getDimName(), d2.getDimName()); } return retVal; } } - ).maximumSize(threshold).create(); + ); + List holderValues = Arrays.asList(holderValueArray); + + // Pull out top aggregated values + final List> values = Lists.transform( + holderValues, + new Function>() + { + @Override + public Map apply(DimValHolder valHolder) + { + return valHolder.getMetricValues(); + } + } + ); + return new Result( + timestamp, + new TopNResultValue(values) + ); } } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index 165e3149456..b52a1c127d4 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -35,6 +35,8 @@ import com.metamx.common.guava.nary.BinaryFn; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.OrderedMergeSequence; import io.druid.granularity.QueryGranularity; +import io.druid.query.BySegmentResultValue; +import io.druid.query.BySegmentResultValueClass; import io.druid.query.CacheStrategy; import io.druid.query.IntervalChunkingQueryRunner; import io.druid.query.Query; @@ -154,6 +156,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest, Result>() { private String dimension = query.getDimensionSpec().getOutputName(); + final List prunedAggs = prunePostAggregators(query); @Override public Result apply(Result result) @@ -170,7 +173,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest apply(Result input) { if (isBySegment) { - BySegmentTopNResultValue value = (BySegmentTopNResultValue) input.getValue(); + BySegmentResultValue> value = (BySegmentResultValue>) input + .getValue(); return new Result( input.getTimestamp(), @@ -460,7 +464,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest chunkEncoderPool = new StupidPool( new Supplier() { @@ -44,7 +45,7 @@ public class CompressedPools public ChunkEncoder get() { log.info("Allocating new chunkEncoder[%,d]", counter.incrementAndGet()); - return new ChunkEncoder(0xFFFF); + return new ChunkEncoder(BUFFER_SIZE); } } ); @@ -63,7 +64,7 @@ public class CompressedPools public byte[] get() { log.info("Allocating new outputBytesPool[%,d]", counter.incrementAndGet()); - return new byte[0xFFFF]; + return new byte[BUFFER_SIZE]; } } ); @@ -82,7 +83,7 @@ public class CompressedPools public ByteBuffer get() { log.info("Allocating new bigEndByteBuf[%,d]", counter.incrementAndGet()); - return ByteBuffer.allocateDirect(0xFFFF).order(ByteOrder.BIG_ENDIAN); + return ByteBuffer.allocateDirect(BUFFER_SIZE).order(ByteOrder.BIG_ENDIAN); } } ); @@ -96,7 +97,7 @@ public class CompressedPools public ByteBuffer get() { log.info("Allocating new littleEndByteBuf[%,d]", counter.incrementAndGet()); - return ByteBuffer.allocateDirect(0xFFFF).order(ByteOrder.LITTLE_ENDIAN); + return ByteBuffer.allocateDirect(BUFFER_SIZE).order(ByteOrder.LITTLE_ENDIAN); } } ); diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 7e1001afb11..bf563080bfa 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -35,6 +35,10 @@ import com.google.common.primitives.Ints; import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Module; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ConciseBitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; +import com.metamx.collections.bitmap.MutableBitmap; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.IAE; import com.metamx.common.ISE; @@ -47,7 +51,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.common.utils.SerializerUtils; import io.druid.guice.ConfigProvider; import io.druid.guice.GuiceInjectors; -import io.druid.jackson.DefaultObjectMapper; +import io.druid.guice.JsonConfigProvider; import io.druid.query.DruidProcessingConfig; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; @@ -55,9 +59,10 @@ import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; import io.druid.segment.data.ArrayIndexed; +import io.druid.segment.data.BitmapSerde; +import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferSerializer; import io.druid.segment.data.CompressedLongsIndexedSupplier; -import io.druid.segment.data.ConciseCompressedIndexedInts; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.IndexedRTree; @@ -73,8 +78,6 @@ import io.druid.segment.serde.FloatGenericColumnSupplier; import io.druid.segment.serde.LongGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnSupplier; import io.druid.segment.serde.SpatialIndexColumnPartSupplier; -import it.uniroma3.mat.extendedset.intset.ConciseSet; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.Interval; import java.io.ByteArrayOutputStream; @@ -120,6 +123,8 @@ public class IndexIO private static final SerializerUtils serializerUtils = new SerializerUtils(); private static final ObjectMapper mapper; + private static final BitmapSerdeFactory bitmapSerdeFactory; + protected static final ColumnConfig columnConfig; static { @@ -136,26 +141,19 @@ public class IndexIO ImmutableMap.of("base_path", "druid.processing") ); binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); + + JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class); } } ) ); mapper = injector.getInstance(ObjectMapper.class); columnConfig = injector.getInstance(ColumnConfig.class); - } - - private static volatile IndexIOHandler handler = null; - - @Deprecated - public static MMappedIndex mapDir(final File inDir) throws IOException - { - init(); - return handler.mapDir(inDir); + bitmapSerdeFactory = injector.getInstance(BitmapSerdeFactory.class); } public static QueryableIndex loadIndex(File inDir) throws IOException { - init(); final int version = SegmentUtils.getVersionFromDir(inDir); final IndexLoader loader = indexLoaders.get(version); @@ -167,27 +165,6 @@ public class IndexIO } } - public static boolean hasHandler() - { - return (IndexIO.handler != null); - } - - public static void registerHandler(IndexIOHandler handler) - { - if (IndexIO.handler == null) { - IndexIO.handler = handler; - } else { - throw new ISE("Already have a handler[%s], cannot register another[%s]", IndexIO.handler, handler); - } - } - - private static void init() - { - if (handler == null) { - handler = new DefaultIndexIOHandler(); - } - } - public static int getVersionFromDir(File inDir) throws IOException { File versionFile = new File(inDir, "version.bin"); @@ -228,7 +205,7 @@ public class IndexIO case 6: case 7: log.info("Old version, re-persisting."); - IndexMaker.append( + IndexMerger.append( Arrays.asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))), converted ); @@ -280,6 +257,7 @@ public class IndexIO indexBuffer, GenericIndexed.stringStrategy ); final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); + final BitmapSerdeFactory bitmapSerdeFactory = new BitmapSerde.LegacyBitmapSerdeFactory(); CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromByteBuffer( smooshedFiles.mapFile(makeTimeFile(inDir, BYTE_ORDER).getName()), BYTE_ORDER @@ -298,7 +276,7 @@ public class IndexIO Map> dimValueLookups = Maps.newHashMap(); Map dimColumns = Maps.newHashMap(); - Map> invertedIndexed = Maps.newHashMap(); + Map> bitmaps = Maps.newHashMap(); for (String dimension : IndexedIterable.create(availableDimensions)) { ByteBuffer dimBuffer = smooshedFiles.mapFile(makeDimFile(inDir, dimension).getName()); @@ -316,9 +294,9 @@ public class IndexIO ByteBuffer invertedBuffer = smooshedFiles.mapFile("inverted.drd"); for (int i = 0; i < availableDimensions.size(); ++i) { - invertedIndexed.put( + bitmaps.put( serializerUtils.readString(invertedBuffer), - GenericIndexed.read(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy) + GenericIndexed.read(invertedBuffer, bitmapSerdeFactory.getObjectStrategy()) ); } @@ -327,7 +305,10 @@ public class IndexIO while (spatialBuffer != null && spatialBuffer.hasRemaining()) { spatialIndexed.put( serializerUtils.readString(spatialBuffer), - ByteBufferSerializer.read(spatialBuffer, IndexedRTree.objectStrategy) + ByteBufferSerializer.read( + spatialBuffer, + new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()) + ) ); } @@ -339,7 +320,7 @@ public class IndexIO metrics, dimValueLookups, dimColumns, - invertedIndexed, + bitmaps, spatialIndexed, smooshedFiles ); @@ -371,13 +352,14 @@ public class IndexIO final FileSmoosher v9Smoosher = new FileSmoosher(v9Dir); ByteStreams.write(Ints.toByteArray(9), Files.newOutputStreamSupplier(new File(v9Dir, "version.bin"))); - Map> bitmapIndexes = Maps.newHashMap(); + Map> bitmapIndexes = Maps.newHashMap(); final ByteBuffer invertedBuffer = v8SmooshedFiles.mapFile("inverted.drd"); while (invertedBuffer.hasRemaining()) { + final String dimName = serializerUtils.readString(invertedBuffer); bitmapIndexes.put( - serializerUtils.readString(invertedBuffer), - GenericIndexed.read(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy) + dimName, + GenericIndexed.read(invertedBuffer, bitmapSerdeFactory.getObjectStrategy()) ); } @@ -386,7 +368,11 @@ public class IndexIO while (spatialBuffer != null && spatialBuffer.hasRemaining()) { spatialIndexes.put( serializerUtils.readString(spatialBuffer), - ByteBufferSerializer.read(spatialBuffer, IndexedRTree.objectStrategy) + ByteBufferSerializer.read( + spatialBuffer, new IndexedRTree.ImmutableRTreeObjectStrategy( + bitmapSerdeFactory.getBitmapFactory() + ) + ) ); } @@ -422,11 +408,12 @@ public class IndexIO VSizeIndexedInts singleValCol = null; VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer()); - GenericIndexed bitmaps = bitmapIndexes.get(dimension); + GenericIndexed bitmaps = bitmapIndexes.get(dimension); ImmutableRTree spatialIndex = spatialIndexes.get(dimension); + final BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); boolean onlyOneValue = true; - ConciseSet nullsSet = null; + MutableBitmap nullsSet = null; for (int i = 0; i < multiValCol.size(); ++i) { VSizeIndexedInts rowValue = multiValCol.get(i); if (!onlyOneValue) { @@ -437,7 +424,7 @@ public class IndexIO } if (rowValue.size() == 0) { if (nullsSet == null) { - nullsSet = new ConciseSet(); + nullsSet = bitmapFactory.makeEmptyMutableBitmap(); } nullsSet.add(i); } @@ -448,7 +435,7 @@ public class IndexIO final boolean bumpedDictionary; if (nullsSet != null) { log.info("Dimension[%s] has null rows.", dimension); - final ImmutableConciseSet theNullSet = ImmutableConciseSet.newImmutableFromMutable(nullsSet); + final ImmutableBitmap theNullSet = bitmapFactory.makeImmutableBitmap(nullsSet); if (dictionary.get(0) != null) { log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension); @@ -463,16 +450,19 @@ public class IndexIO bitmaps = GenericIndexed.fromIterable( Iterables.concat(Arrays.asList(theNullSet), bitmaps), - ConciseCompressedIndexedInts.objectStrategy + bitmapSerdeFactory.getObjectStrategy() ); } else { bumpedDictionary = false; bitmaps = GenericIndexed.fromIterable( Iterables.concat( - Arrays.asList(ImmutableConciseSet.union(theNullSet, bitmaps.get(0))), + Arrays.asList( + bitmapFactory + .union(Arrays.asList(theNullSet, bitmaps.get(0))) + ), Iterables.skip(bitmaps, 1) ), - ConciseCompressedIndexedInts.objectStrategy + bitmapSerdeFactory.getObjectStrategy() ); } } else { @@ -508,6 +498,7 @@ public class IndexIO dictionary, singleValCol, multiValCol, + bitmapSerdeFactory, bitmaps, spatialIndex ) @@ -606,13 +597,13 @@ public class IndexIO final GenericIndexed dims9 = GenericIndexed.fromIterable( Iterables.filter( dims8, new Predicate() - { - @Override - public boolean apply(String s) - { - return !skippedDimensions.contains(s); - } - } + { + @Override + public boolean apply(String s) + { + return !skippedDimensions.contains(s); + } + } ), GenericIndexed.stringStrategy ); @@ -620,6 +611,10 @@ public class IndexIO indexBuffer, GenericIndexed.stringStrategy ); final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); + final BitmapSerdeFactory segmentBitmapSerdeFactory = mapper.readValue( + serializerUtils.readString(indexBuffer), + BitmapSerdeFactory.class + ); Set columns = Sets.newTreeSet(); columns.addAll(Lists.newArrayList(dims9)); @@ -627,12 +622,17 @@ public class IndexIO GenericIndexed cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy); - final long numBytes = cols.getSerializedSize() + dims9.getSerializedSize() + 16; + final String segmentBitmapSerdeFactoryString = mapper.writeValueAsString(segmentBitmapSerdeFactory); + + final long numBytes = cols.getSerializedSize() + dims9.getSerializedSize() + 16 + + serializerUtils.getSerializedStringByteSize(segmentBitmapSerdeFactoryString); + final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); cols.writeToChannel(writer); dims9.writeToChannel(writer); serializerUtils.writeLong(writer, dataInterval.getStartMillis()); serializerUtils.writeLong(writer, dataInterval.getEndMillis()); + serializerUtils.writeString(writer, segmentBitmapSerdeFactoryString); writer.close(); log.info("Skipped files[%s]", skippedFiles); @@ -648,10 +648,12 @@ public class IndexIO static class LegacyIndexLoader implements IndexLoader { + private static final IndexIOHandler legacyHandler = new DefaultIndexIOHandler(); + @Override public QueryableIndex load(File inDir) throws IOException { - MMappedIndex index = IndexIO.mapDir(inDir); + MMappedIndex index = legacyHandler.mapDir(inDir); Map columns = Maps.newHashMap(); @@ -669,7 +671,9 @@ public class IndexIO ) .setBitmapIndex( new BitmapIndexColumnPartSupplier( - index.getInvertedIndexes().get(dimension), index.getDimValueLookup(dimension) + new ConciseBitmapFactory(), + index.getBitmapIndexes().get(dimension), + index.getDimValueLookup(dimension) ) ); if (index.getSpatialIndexes().get(dimension) != null) { @@ -719,14 +723,17 @@ public class IndexIO } String[] cols = colSet.toArray(new String[colSet.size()]); - columns.put(Column.TIME_COLUMN_NAME, new ColumnBuilder() - .setType(ValueType.LONG) - .setGenericColumn(new LongGenericColumnSupplier(index.timestamps)) - .build()); + columns.put( + Column.TIME_COLUMN_NAME, new ColumnBuilder() + .setType(ValueType.LONG) + .setGenericColumn(new LongGenericColumnSupplier(index.timestamps)) + .build() + ); return new SimpleQueryableIndex( index.getDataInterval(), new ArrayIndexed<>(cols, String.class), index.getAvailableDimensions(), + new ConciseBitmapFactory(), columns, index.getFileMapper() ); @@ -749,9 +756,24 @@ public class IndexIO SmooshedFileMapper smooshedFiles = Smoosh.map(inDir); ByteBuffer indexBuffer = smooshedFiles.mapFile("index.drd"); + /** + * Index.drd should consist of the segment version, the columns and dimensions of the segment as generic + * indexes, the interval start and end millis as longs (in 16 bytes), and a bitmap index type. + */ final GenericIndexed cols = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy); final GenericIndexed dims = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy); final Interval dataInterval = new Interval(indexBuffer.getLong(), indexBuffer.getLong()); + final BitmapSerdeFactory segmentBitmapSerdeFactory; + /** + * This is a workaround for the fact that in v8 segments, we have no information about the type of bitmap + * index to use. Since we cannot very cleanly build v9 segments directly, we are using a workaround where + * this information is appended to the end of index.drd. + */ + if (indexBuffer.hasRemaining()) { + segmentBitmapSerdeFactory = mapper.readValue(serializerUtils.readString(indexBuffer), BitmapSerdeFactory.class); + } else { + segmentBitmapSerdeFactory = new BitmapSerde.LegacyBitmapSerdeFactory(); + } Map columns = Maps.newHashMap(); @@ -762,7 +784,7 @@ public class IndexIO columns.put(Column.TIME_COLUMN_NAME, deserializeColumn(mapper, smooshedFiles.mapFile("__time"))); final QueryableIndex index = new SimpleQueryableIndex( - dataInterval, cols, dims, columns, smooshedFiles + dataInterval, cols, dims, segmentBitmapSerdeFactory.getBitmapFactory(), columns, smooshedFiles ); log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime); diff --git a/processing/src/main/java/io/druid/segment/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java index d054e0bedc2..8b304e77605 100644 --- a/processing/src/main/java/io/druid/segment/IndexMaker.java +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -24,6 +24,7 @@ import com.google.common.base.Function; import com.google.common.base.Objects; import com.google.common.base.Predicate; import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; @@ -33,7 +34,12 @@ import com.google.common.collect.Sets; import com.google.common.io.ByteStreams; import com.google.common.io.Files; import com.google.common.primitives.Ints; +import com.google.inject.Binder; import com.google.inject.Injector; +import com.google.inject.Module; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; +import com.metamx.collections.bitmap.MutableBitmap; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.RTree; import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; @@ -46,21 +52,20 @@ import com.metamx.common.io.smoosh.FileSmoosher; import com.metamx.common.io.smoosh.SmooshedWriter; import com.metamx.common.logger.Logger; import io.druid.collections.CombiningIterable; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidPool; import io.druid.common.utils.JodaUtils; import io.druid.common.utils.SerializerUtils; import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.ToLowerCaseAggregatorFactory; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; +import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.CompressedFloatsIndexedSupplier; import io.druid.segment.data.CompressedLongsIndexedSupplier; import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.ConciseCompressedIndexedInts; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; @@ -76,15 +81,12 @@ import io.druid.segment.serde.ComplexMetrics; import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; import io.druid.segment.serde.FloatGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnPartSerde; -import it.uniroma3.mat.extendedset.intset.ConciseSet; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.ByteArrayOutputStream; -import java.io.Closeable; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -109,10 +111,23 @@ public class IndexMaker private static final int INVALID_ROW = -1; private static final Splitter SPLITTER = Splitter.on(","); private static final ObjectMapper mapper; + private static final BitmapSerdeFactory bitmapSerdeFactory; static { - final Injector injector = GuiceInjectors.makeStartupInjector(); + final Injector injector = GuiceInjectors.makeStartupInjectorWithModules( + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class); + } + } + ) + ); mapper = injector.getInstance(ObjectMapper.class); + bitmapSerdeFactory = injector.getInstance(BitmapSerdeFactory.class); } public static File persist(final IncrementalIndex index, File outDir) throws IOException @@ -170,7 +185,7 @@ public class IndexMaker log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); return merge( - Arrays.asList(new IncrementalIndexAdapter(dataInterval, index)), + Arrays.asList(new IncrementalIndexAdapter(dataInterval, index, bitmapSerdeFactory.getBitmapFactory())), index.getMetricAggs(), outDir, progress @@ -839,13 +854,14 @@ public class IndexMaker ? new MultiValColumnDictionaryEntryStore() : new SingleValColumnDictionaryEntryStore(); - ConciseSet nullSet = null; + final BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); + MutableBitmap nullSet = null; int rowCount = 0; for (Rowboat theRow : theRows) { if (dimIndex > theRow.getDims().length) { if (nullSet == null) { - nullSet = new ConciseSet(); + nullSet = bitmapFactory.makeEmptyMutableBitmap(); } nullSet.add(rowCount); adder.add(null); @@ -853,7 +869,7 @@ public class IndexMaker int[] dimVals = theRow.getDims()[dimIndex]; if (dimVals == null || dimVals.length == 0) { if (nullSet == null) { - nullSet = new ConciseSet(); + nullSet = bitmapFactory.makeEmptyMutableBitmap(); } nullSet.add(rowCount); } @@ -1062,18 +1078,18 @@ public class IndexMaker } // Make bitmap indexes - List conciseSets = Lists.newArrayList(); + List mutableBitmaps = Lists.newArrayList(); for (String dimVal : dimensionValues) { List> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size()); for (int j = 0; j < adapters.size(); ++j) { convertedInverteds.add( new ConvertingIndexedInts( - adapters.get(j).getInverteds(dimension, dimVal), rowNumConversions.get(j) + adapters.get(j).getBitmapIndex(dimension, dimVal), rowNumConversions.get(j) ) ); } - ConciseSet bitset = new ConciseSet(); + MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap(); for (Integer row : CombiningIterable.createSplatted( convertedInverteds, Ordering.natural().nullsFirst() @@ -1083,40 +1099,40 @@ public class IndexMaker } } - conciseSets.add(bitset); + mutableBitmaps.add(bitset); } - GenericIndexed bitmaps; + GenericIndexed bitmaps; if (nullSet != null) { - final ImmutableConciseSet theNullSet = ImmutableConciseSet.newImmutableFromMutable(nullSet); + final ImmutableBitmap theNullSet = bitmapFactory.makeImmutableBitmap(nullSet); if (bumpDictionary) { bitmaps = GenericIndexed.fromIterable( Iterables.concat( Arrays.asList(theNullSet), Iterables.transform( - conciseSets, - new Function() + mutableBitmaps, + new Function() { @Override - public ImmutableConciseSet apply(ConciseSet input) + public ImmutableBitmap apply(MutableBitmap input) { - return ImmutableConciseSet.newImmutableFromMutable(input); + return bitmapFactory.makeImmutableBitmap(input); } } ) ), - ConciseCompressedIndexedInts.objectStrategy + bitmapSerdeFactory.getObjectStrategy() ); } else { - Iterable immutableConciseSets = Iterables.transform( - conciseSets, - new Function() + Iterable immutableBitmaps = Iterables.transform( + mutableBitmaps, + new Function() { @Override - public ImmutableConciseSet apply(ConciseSet input) + public ImmutableBitmap apply(MutableBitmap input) { - return ImmutableConciseSet.newImmutableFromMutable(input); + return bitmapFactory.makeImmutableBitmap(input); } } ); @@ -1124,30 +1140,27 @@ public class IndexMaker bitmaps = GenericIndexed.fromIterable( Iterables.concat( Arrays.asList( - ImmutableConciseSet.union( - theNullSet, - Iterables.getFirst(immutableConciseSets, null) - ) + theNullSet.union(Iterables.getFirst(immutableBitmaps, null)) ), - Iterables.skip(immutableConciseSets, 1) + Iterables.skip(immutableBitmaps, 1) ), - ConciseCompressedIndexedInts.objectStrategy + bitmapSerdeFactory.getObjectStrategy() ); } } else { bitmaps = GenericIndexed.fromIterable( Iterables.transform( - conciseSets, - new Function() + mutableBitmaps, + new Function() { @Override - public ImmutableConciseSet apply(ConciseSet input) + public ImmutableBitmap apply(MutableBitmap input) { - return ImmutableConciseSet.newImmutableFromMutable(input); + return bitmapFactory.makeImmutableBitmap(input); } } ), - ConciseCompressedIndexedInts.objectStrategy + bitmapSerdeFactory.getObjectStrategy() ); } @@ -1156,7 +1169,11 @@ public class IndexMaker boolean hasSpatialIndexes = columnCapabilities.get(dimension).hasSpatialIndexes(); RTree tree = null; if (hasSpatialIndexes) { - tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50)); + tree = new RTree( + 2, + new LinearGutmanSplitStrategy(0, 50, bitmapSerdeFactory.getBitmapFactory()), + bitmapSerdeFactory.getBitmapFactory() + ); } int dimValIndex = 0; @@ -1168,7 +1185,7 @@ public class IndexMaker for (int j = 0; j < coords.length; j++) { coords[j] = Float.valueOf(stringCoords.get(j)); } - tree.insert(coords, conciseSets.get(dimValIndex)); + tree.insert(coords, mutableBitmaps.get(dimValIndex)); } dimValIndex++; } @@ -1185,6 +1202,7 @@ public class IndexMaker dictionary, singleValCol, multiValCol, + bitmapSerdeFactory, bitmaps, spatialIndex ), @@ -1354,7 +1372,12 @@ public class IndexMaker GenericIndexed cols = GenericIndexed.fromIterable(finalColumns, GenericIndexed.stringStrategy); GenericIndexed dims = GenericIndexed.fromIterable(finalDimensions, GenericIndexed.stringStrategy); - final long numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; + final String bitmapSerdeFactoryType = mapper.writeValueAsString(bitmapSerdeFactory); + final long numBytes = cols.getSerializedSize() + + dims.getSerializedSize() + + 16 + + serializerUtils.getSerializedStringByteSize(bitmapSerdeFactoryType); + final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); cols.writeToChannel(writer); @@ -1371,6 +1394,9 @@ public class IndexMaker serializerUtils.writeLong(writer, dataInterval.getStartMillis()); serializerUtils.writeLong(writer, dataInterval.getEndMillis()); + serializerUtils.writeString( + writer, bitmapSerdeFactoryType + ); writer.close(); IndexIO.checkFileSize(new File(outDir, "index.drd")); diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index d5947b21cab..e21ae50e8f4 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -19,9 +19,11 @@ package io.druid.segment; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Objects; import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; @@ -32,6 +34,12 @@ import com.google.common.io.ByteStreams; import com.google.common.io.Files; import com.google.common.io.OutputSupplier; import com.google.common.primitives.Ints; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; +import com.metamx.collections.bitmap.MutableBitmap; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.RTree; import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; @@ -48,15 +56,17 @@ import io.druid.common.guava.FileOutputSupplier; import io.druid.common.guava.GuavaUtils; import io.druid.common.utils.JodaUtils; import io.druid.common.utils.SerializerUtils; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.ToLowerCaseAggregatorFactory; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ValueType; +import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.CompressedLongsSupplierSerializer; import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.ConciseCompressedIndexedInts; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.IOPeon; @@ -71,8 +81,6 @@ import io.druid.segment.incremental.IncrementalIndexAdapter; import io.druid.segment.serde.ComplexMetricColumnSerializer; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; -import it.uniroma3.mat.extendedset.intset.ConciseSet; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -103,6 +111,27 @@ public class IndexMerger private static final int INVALID_ROW = -1; private static final Splitter SPLITTER = Splitter.on(","); + private static final ObjectMapper mapper; + private static final BitmapSerdeFactory bitmapSerdeFactory; + + static { + final Injector injector = GuiceInjectors.makeStartupInjectorWithModules( + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class); + } + } + ) + ); + mapper = injector.getInstance(ObjectMapper.class); + bitmapSerdeFactory = injector.getInstance(BitmapSerdeFactory.class); + } + + public static File persist(final IncrementalIndex index, File outDir) throws IOException { return persist(index, index.getInterval(), outDir); @@ -153,7 +182,13 @@ public class IndexMerger log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); return merge( - Arrays.asList(new IncrementalIndexAdapter(dataInterval, index)), + Arrays.asList( + new IncrementalIndexAdapter( + dataInterval, + index, + bitmapSerdeFactory.getBitmapFactory() + ) + ), index.getMetricAggs(), outDir, progress @@ -458,6 +493,7 @@ public class IndexMerger dataInterval = new Interval(minTime, maxTime); serializerUtils.writeString(channel, String.format("%s/%s", minTime, maxTime)); + serializerUtils.writeString(channel, mapper.writeValueAsString(bitmapSerdeFactory)); } finally { CloseQuietly.close(channel); @@ -756,8 +792,8 @@ public class IndexMerger Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.stringStrategy); log.info("Starting dimension[%s] with cardinality[%,d]", dimension, dimVals.size()); - GenericIndexedWriter writer = new GenericIndexedWriter( - ioPeon, dimension, ConciseCompressedIndexedInts.objectStrategy + GenericIndexedWriter writer = new GenericIndexedWriter<>( + ioPeon, dimension, bitmapSerdeFactory.getObjectStrategy() ); writer.open(); @@ -766,11 +802,12 @@ public class IndexMerger RTree tree = null; IOPeon spatialIoPeon = new TmpFileIOPeon(); if (isSpatialDim) { + BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); spatialWriter = new ByteBufferWriter( - spatialIoPeon, dimension, IndexedRTree.objectStrategy + spatialIoPeon, dimension, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapFactory) ); spatialWriter.open(); - tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50)); + tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory); } for (String dimVal : IndexedIterable.create(dimVals)) { @@ -779,12 +816,12 @@ public class IndexMerger for (int j = 0; j < indexes.size(); ++j) { convertedInverteds.add( new ConvertingIndexedInts( - indexes.get(j).getInverteds(dimension, dimVal), rowNumConversions.get(j) + indexes.get(j).getBitmapIndex(dimension, dimVal), rowNumConversions.get(j) ) ); } - ConciseSet bitset = new ConciseSet(); + MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap(); for (Integer row : CombiningIterable.createSplatted( convertedInverteds, Ordering.natural().nullsFirst() @@ -794,7 +831,9 @@ public class IndexMerger } } - writer.write(ImmutableConciseSet.newImmutableFromMutable(bitset)); + writer.write( + bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset) + ); if (isSpatialDim && dimVal != null) { List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); @@ -906,6 +945,9 @@ public class IndexMerger serializerUtils.writeString( channel, String.format("%s/%s", dataInterval.getStart(), dataInterval.getEnd()) ); + serializerUtils.writeString( + channel, mapper.writeValueAsString(bitmapSerdeFactory) + ); } finally { CloseQuietly.close(channel); diff --git a/processing/src/main/java/io/druid/segment/IndexableAdapter.java b/processing/src/main/java/io/druid/segment/IndexableAdapter.java index e676cee6e19..418a1683d14 100644 --- a/processing/src/main/java/io/druid/segment/IndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/IndexableAdapter.java @@ -42,7 +42,7 @@ public interface IndexableAdapter Iterable getRows(); - IndexedInts getInverteds(String dimension, String value); + IndexedInts getBitmapIndex(String dimension, String value); String getMetricType(String metric); diff --git a/processing/src/main/java/io/druid/segment/MMappedIndex.java b/processing/src/main/java/io/druid/segment/MMappedIndex.java index 5f422ac1a84..5634b0ceec1 100644 --- a/processing/src/main/java/io/druid/segment/MMappedIndex.java +++ b/processing/src/main/java/io/druid/segment/MMappedIndex.java @@ -19,13 +19,13 @@ package io.druid.segment; -import com.google.common.collect.Maps; +import com.metamx.collections.bitmap.ImmutableBitmap; +import com.metamx.collections.bitmap.WrappedImmutableConciseBitmap; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.io.smoosh.SmooshedFileMapper; import com.metamx.common.logger.Logger; import io.druid.segment.data.CompressedLongsIndexedSupplier; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.IndexedLongs; import io.druid.segment.data.VSizeIndexed; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.Interval; @@ -38,7 +38,6 @@ import java.util.Map; public class MMappedIndex { private static final Logger log = new Logger(MMappedIndex.class); - private static final ImmutableConciseSet emptySet = new ImmutableConciseSet(); final GenericIndexed availableDimensions; final GenericIndexed availableMetrics; @@ -47,12 +46,10 @@ public class MMappedIndex final Map metrics; final Map> dimValueLookups; final Map dimColumns; - final Map> invertedIndexes; + final Map> invertedIndexes; final Map spatialIndexes; final SmooshedFileMapper fileMapper; - private final Map metricIndexes = Maps.newHashMap(); - public MMappedIndex( GenericIndexed availableDimensions, GenericIndexed availableMetrics, @@ -61,7 +58,7 @@ public class MMappedIndex Map metrics, Map> dimValueLookups, Map dimColumns, - Map> invertedIndexes, + Map> invertedIndexes, Map spatialIndexes, SmooshedFileMapper fileMapper ) @@ -76,10 +73,6 @@ public class MMappedIndex this.invertedIndexes = invertedIndexes; this.spatialIndexes = spatialIndexes; this.fileMapper = fileMapper; - - for (int i = 0; i < availableMetrics.size(); i++) { - metricIndexes.put(availableMetrics.get(i), i); - } } public CompressedLongsIndexedSupplier getTimestamps() @@ -102,21 +95,11 @@ public class MMappedIndex return metrics; } - public Integer getMetricIndex(String metricName) - { - return metricIndexes.get(metricName); - } - public Interval getDataInterval() { return dataInterval; } - public IndexedLongs getReadOnlyTimestamps() - { - return timestamps.get(); - } - public MetricHolder getMetricHolder(String metric) { final MetricHolder retVal = metrics.get(metric); @@ -138,7 +121,7 @@ public class MMappedIndex return dimColumns.get(dimension); } - public Map> getInvertedIndexes() + public Map> getBitmapIndexes() { return invertedIndexes; } @@ -148,22 +131,6 @@ public class MMappedIndex return spatialIndexes; } - public ImmutableConciseSet getInvertedIndex(String dimension, String value) - { - final GenericIndexed lookup = dimValueLookups.get(dimension); - if (lookup == null) { - return emptySet; - } - - int indexOf = lookup.indexOf(value); - if (indexOf < 0) { - return emptySet; - } - - ImmutableConciseSet retVal = invertedIndexes.get(dimension).get(indexOf); - return (retVal == null) ? emptySet : retVal; - } - public SmooshedFileMapper getFileMapper() { return fileMapper; diff --git a/processing/src/main/java/io/druid/segment/QueryableIndex.java b/processing/src/main/java/io/druid/segment/QueryableIndex.java index 7de331e80e3..acef8c7e321 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndex.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndex.java @@ -17,7 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;import io.druid.segment.data.Indexed; +package io.druid.segment; + +import com.metamx.collections.bitmap.BitmapFactory; +import io.druid.segment.data.Indexed; import org.joda.time.Interval; import java.io.IOException; @@ -30,6 +33,7 @@ public interface QueryableIndex extends ColumnSelector public int getNumRows(); public Indexed getColumnNames(); public Indexed getAvailableDimensions(); + public BitmapFactory getBitmapFactoryForDimensions(); /** * The close method shouldn't actually be here as this is nasty. We will adjust it in the future. diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index 6b20722360f..5f96512f6e3 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -35,7 +35,7 @@ import io.druid.segment.column.IndexedFloatsGenericColumn; import io.druid.segment.column.IndexedLongsGenericColumn; import io.druid.segment.column.ValueType; import io.druid.segment.data.ArrayBasedIndexedInts; -import io.druid.segment.data.ConciseCompressedIndexedInts; +import io.druid.segment.data.BitmapCompressedIndexedInts; import io.druid.segment.data.EmptyIndexedInts; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; @@ -279,7 +279,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter } @Override - public IndexedInts getInverteds(String dimension, String value) + public IndexedInts getBitmapIndex(String dimension, String value) { final Column column = input.getColumn(dimension); @@ -292,7 +292,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter return new EmptyIndexedInts(); } - return new ConciseCompressedIndexedInts(bitmaps.getConciseSet(value)); + return new BitmapCompressedIndexedInts(bitmaps.getBitmap(value)); } @Override diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 3af73449ec5..9d0808aeedf 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -154,16 +154,22 @@ public class QueryableIndexStorageAdapter implements StorageAdapter actualInterval = actualInterval.withEnd(dataInterval.getEnd()); } - final Sequence sequence; + final Offset offset; if (filter == null) { - sequence = new NoFilterCursorSequenceBuilder(index, actualInterval, gran).build(); + offset = new NoFilterOffset(0, index.getNumRows()); } else { - Offset offset = new ConciseOffset(filter.goConcise(new ColumnSelectorBitmapIndexSelector(index))); + final ColumnSelectorBitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector( + index.getBitmapFactoryForDimensions(), + index + ); - sequence = new CursorSequenceBuilder(index, actualInterval, gran, offset).build(); + offset = new BitmapOffset(selector.getBitmapFactory(), filter.getBitmapIndex(selector)); } - return Sequences.filter(sequence, Predicates.notNull()); + return Sequences.filter( + new CursorSequenceBuilder(index, actualInterval, gran, offset).build(), + Predicates.notNull() + ); } private static class CursorSequenceBuilder @@ -267,8 +273,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter final Column columnDesc = index.getColumn(dimensionName); if (cachedColumn == null && columnDesc != null) { - cachedColumn = columnDesc.getDictionaryEncoding(); - dictionaryColumnCache.put(dimensionName, cachedColumn); + cachedColumn = columnDesc.getDictionaryEncoding(); + dictionaryColumnCache.put(dimensionName, cachedColumn); } final DictionaryEncodedColumn column = cachedColumn; @@ -539,7 +545,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter return columnVals.lookupName(multiValueRow.get(0)); } else { final String[] strings = new String[multiValueRow.size()]; - for (int i = 0 ; i < multiValueRow.size() ; i++) { + for (int i = 0; i < multiValueRow.size(); i++) { strings[i] = columnVals.lookupName(multiValueRow.get(i)); } return strings; @@ -600,7 +606,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter CloseQuietly.close(complexColumn); } for (Object column : objectColumnCache.values()) { - if(column instanceof Closeable) { + if (column instanceof Closeable) { CloseQuietly.close((Closeable) column); } } @@ -615,6 +621,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter private final Offset baseOffset; private final GenericColumn timestamps; private final long threshold; + private final boolean allWithinThreshold; public TimestampCheckingOffset( Offset baseOffset, @@ -625,6 +632,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter this.baseOffset = baseOffset; this.timestamps = timestamps; this.threshold = threshold; + // checks if all the values are within the Threshold specified, skips timestamp lookups and checks if all values are within threshold. + this.allWithinThreshold = timestamps.getLongSingleValueRow(timestamps.length() - 1) < threshold; } @Override @@ -642,7 +651,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter @Override public boolean withinBounds() { - return baseOffset.withinBounds() && timestamps.getLongSingleValueRow(baseOffset.getOffset()) < threshold; + return baseOffset.withinBounds() && (allWithinThreshold + || timestamps.getLongSingleValueRow(baseOffset.getOffset()) < threshold); } @Override @@ -652,441 +662,39 @@ public class QueryableIndexStorageAdapter implements StorageAdapter } } - private static class NoFilterCursorSequenceBuilder + private static class NoFilterOffset implements Offset { - private final ColumnSelector index; - private final Interval interval; - private final QueryGranularity gran; + private final int rowCount; + private volatile int currentOffset; - public NoFilterCursorSequenceBuilder( - ColumnSelector index, - Interval interval, - QueryGranularity gran - ) + NoFilterOffset(int currentOffset, int rowCount) { - this.index = index; - this.interval = interval; - this.gran = gran; + this.currentOffset = currentOffset; + this.rowCount = rowCount; } - /** - * This produces iterators of Cursor objects that must be fully processed (until isDone() returns true) before the - * next Cursor is processed. It is *not* safe to pass these cursors off to another thread for parallel processing - * - * @return - */ - public Sequence build() + @Override + public void increment() { - final Map dictionaryColumnCache = Maps.newHashMap(); - final Map genericColumnCache = Maps.newHashMap(); - final Map complexColumnCache = Maps.newHashMap(); - final Map objectColumnCache = Maps.newHashMap(); + currentOffset++; + } - final GenericColumn timestamps = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn(); + @Override + public boolean withinBounds() + { + return currentOffset < rowCount; + } - return Sequences.withBaggage( - Sequences.map( - Sequences.simple(gran.iterable(interval.getStartMillis(), interval.getEndMillis())), - new Function() - { - private int currRow = 0; + @Override + public Offset clone() + { + return new NoFilterOffset(currentOffset, rowCount); + } - @Override - public Cursor apply(final Long input) - { - final long timeStart = Math.max(interval.getStartMillis(), input); - while (currRow < timestamps.length() && timestamps.getLongSingleValueRow(currRow) < timeStart) { - ++currRow; - } - - return new Cursor() - { - private final DateTime myBucket = gran.toDateTime(input); - private final long nextBucket = Math.min(gran.next(myBucket.getMillis()), interval.getEndMillis()); - private final int initRow = currRow; - - @Override - public DateTime getTime() - { - return myBucket; - } - - @Override - public void advance() - { - if (Thread.interrupted()) { - throw new QueryInterruptedException(); - } - ++currRow; - } - - @Override - public void advanceTo(int offset) - { - currRow += offset; - } - - @Override - public boolean isDone() - { - return currRow >= timestamps.length() || timestamps.getLongSingleValueRow(currRow) >= nextBucket; - } - - @Override - public void reset() - { - currRow = initRow; - } - - @Override - public DimensionSelector makeDimensionSelector(String dimension) - { - final String dimensionName = dimension.toLowerCase(); - - DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimensionName); - final Column columnDesc = index.getColumn(dimensionName); - - if (cachedColumn == null && columnDesc != null) { - cachedColumn = columnDesc.getDictionaryEncoding(); - dictionaryColumnCache.put(dimensionName, cachedColumn); - } - - final DictionaryEncodedColumn column = cachedColumn; - - if (column == null) { - return null; - } else if (columnDesc.getCapabilities().hasMultipleValues()) { - return new DimensionSelector() - { - @Override - public IndexedInts getRow() - { - return column.getMultiValueRow(currRow); - } - - @Override - public int getValueCardinality() - { - return column.getCardinality(); - } - - @Override - public String lookupName(int id) - { - final String retVal = column.lookupName(id); - return retVal == null ? "" : retVal; - } - - @Override - public int lookupId(String name) - { - return column.lookupId(name); - } - }; - } else { - return new DimensionSelector() - { - @Override - public IndexedInts getRow() - { - // using an anonymous class is faster than creating a class that stores a copy of the value - return new IndexedInts() - { - @Override - public int size() - { - return 1; - } - - @Override - public int get(int index) - { - return column.getSingleValueRow(currRow); - } - - @Override - public Iterator iterator() - { - return Iterators.singletonIterator(column.getSingleValueRow(currRow)); - } - }; - } - - @Override - public int getValueCardinality() - { - return column.getCardinality(); - } - - @Override - public String lookupName(int id) - { - return column.lookupName(id); - } - - @Override - public int lookupId(String name) - { - return column.lookupId(name); - } - }; - } - } - - @Override - public FloatColumnSelector makeFloatColumnSelector(String columnName) - { - final String metricName = columnName.toLowerCase(); - GenericColumn cachedMetricVals = genericColumnCache.get(metricName); - - if (cachedMetricVals == null) { - Column holder = index.getColumn(metricName); - if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG - || holder.getCapabilities().getType() == ValueType.FLOAT)) { - cachedMetricVals = holder.getGenericColumn(); - genericColumnCache.put(metricName, cachedMetricVals); - } - } - - if (cachedMetricVals == null) { - return new FloatColumnSelector() - { - @Override - public float get() - { - return 0.0f; - } - }; - } - - final GenericColumn metricVals = cachedMetricVals; - return new FloatColumnSelector() - { - @Override - public float get() - { - return metricVals.getFloatSingleValueRow(currRow); - } - }; - } - - @Override - public LongColumnSelector makeLongColumnSelector(String columnName) - { - final String metricName = columnName.toLowerCase(); - GenericColumn cachedMetricVals = genericColumnCache.get(metricName); - - if (cachedMetricVals == null) { - Column holder = index.getColumn(metricName); - if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG - || holder.getCapabilities().getType() == ValueType.FLOAT)) { - cachedMetricVals = holder.getGenericColumn(); - genericColumnCache.put(metricName, cachedMetricVals); - } - } - - if (cachedMetricVals == null) { - return new LongColumnSelector() - { - @Override - public long get() - { - return 0L; - } - }; - } - - final GenericColumn metricVals = cachedMetricVals; - return new LongColumnSelector() - { - @Override - public long get() - { - return metricVals.getLongSingleValueRow(currRow); - } - }; - } - - @Override - public ObjectColumnSelector makeObjectColumnSelector(String column) - { - final String columnName = column.toLowerCase(); - - Object cachedColumnVals = objectColumnCache.get(columnName); - - if (cachedColumnVals == null) { - Column holder = index.getColumn(columnName); - - if (holder != null) { - final ValueType type = holder.getCapabilities().getType(); - - if (holder.getCapabilities().isDictionaryEncoded()) { - cachedColumnVals = holder.getDictionaryEncoding(); - } else if (type == ValueType.COMPLEX) { - cachedColumnVals = holder.getComplexColumn(); - } else { - cachedColumnVals = holder.getGenericColumn(); - } - } - - if (cachedColumnVals != null) { - objectColumnCache.put(columnName, cachedColumnVals); - } - } - - if (cachedColumnVals == null) { - return null; - } - - if (cachedColumnVals instanceof GenericColumn) { - final GenericColumn columnVals = (GenericColumn) cachedColumnVals; - final ValueType type = columnVals.getType(); - - if (columnVals.hasMultipleValues()) { - throw new UnsupportedOperationException( - "makeObjectColumnSelector does not support multivalued GenericColumns" - ); - } - - if (type == ValueType.FLOAT) { - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return Float.TYPE; - } - - @Override - public Float get() - { - return columnVals.getFloatSingleValueRow(currRow); - } - }; - } - if (type == ValueType.LONG) { - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return Long.TYPE; - } - - @Override - public Long get() - { - return columnVals.getLongSingleValueRow(currRow); - } - }; - } - if (type == ValueType.STRING) { - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return String.class; - } - - @Override - public String get() - { - return columnVals.getStringSingleValueRow(currRow); - } - }; - } - } - - if (cachedColumnVals instanceof DictionaryEncodedColumn) { - final DictionaryEncodedColumn columnVals = (DictionaryEncodedColumn) cachedColumnVals; - if (columnVals.hasMultipleValues()) { - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return Object.class; - } - - @Override - public Object get() - { - final IndexedInts multiValueRow = columnVals.getMultiValueRow(currRow); - if (multiValueRow.size() == 0) { - return null; - } else if (multiValueRow.size() == 1) { - return columnVals.lookupName(multiValueRow.get(0)); - } else { - final String[] strings = new String[multiValueRow.size()]; - for (int i = 0 ; i < multiValueRow.size() ; i++) { - strings[i] = columnVals.lookupName(multiValueRow.get(i)); - } - return strings; - } - } - }; - } else { - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return String.class; - } - - @Override - public String get() - { - return columnVals.lookupName(columnVals.getSingleValueRow(currRow)); - } - }; - } - } - - final ComplexColumn columnVals = (ComplexColumn) cachedColumnVals; - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return columnVals.getClazz(); - } - - @Override - public Object get() - { - return columnVals.getRowValue(currRow); - } - }; - } - }; - } - } - ), - new Closeable() - { - @Override - public void close() throws IOException - { - CloseQuietly.close(timestamps); - for (DictionaryEncodedColumn column : dictionaryColumnCache.values()) { - CloseQuietly.close(column); - } - for (GenericColumn column : genericColumnCache.values()) { - CloseQuietly.close(column); - } - for (ComplexColumn complexColumn : complexColumnCache.values()) { - CloseQuietly.close(complexColumn); - } - for (Object column : objectColumnCache.values()) { - if (column instanceof Closeable) { - CloseQuietly.close((Closeable) column); - } - } - } - } - ); + @Override + public int getOffset() + { + return currentOffset; } } } diff --git a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java index b2d122098f9..7d0958df995 100644 --- a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java @@ -76,9 +76,9 @@ public class RowboatFilteringIndexAdapter implements IndexableAdapter } @Override - public IndexedInts getInverteds(String dimension, String value) + public IndexedInts getBitmapIndex(String dimension, String value) { - return baseAdapter.getInverteds(dimension, value); + return baseAdapter.getBitmapIndex(dimension, value); } @Override diff --git a/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java index c2b569dda78..0b65f6e2bd8 100644 --- a/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java @@ -20,6 +20,7 @@ package io.druid.segment; import com.google.common.base.Preconditions; +import com.metamx.collections.bitmap.BitmapFactory; import com.metamx.common.io.smoosh.SmooshedFileMapper; import io.druid.segment.column.Column; import io.druid.segment.data.Indexed; @@ -35,6 +36,7 @@ public class SimpleQueryableIndex implements QueryableIndex private final Interval dataInterval; private final Indexed columnNames; private final Indexed availableDimensions; + private final BitmapFactory bitmapFactory; private final Map columns; private final SmooshedFileMapper fileMapper; @@ -42,6 +44,7 @@ public class SimpleQueryableIndex implements QueryableIndex Interval dataInterval, Indexed columnNames, Indexed dimNames, + BitmapFactory bitmapFactory, Map columns, SmooshedFileMapper fileMapper ) @@ -50,6 +53,7 @@ public class SimpleQueryableIndex implements QueryableIndex this.dataInterval = dataInterval; this.columnNames = columnNames; this.availableDimensions = dimNames; + this.bitmapFactory = bitmapFactory; this.columns = columns; this.fileMapper = fileMapper; } @@ -78,6 +82,12 @@ public class SimpleQueryableIndex implements QueryableIndex return availableDimensions; } + @Override + public BitmapFactory getBitmapFactoryForDimensions() + { + return bitmapFactory; + } + @Override public Column getColumn(String columnName) { diff --git a/processing/src/main/java/io/druid/segment/column/BitmapIndex.java b/processing/src/main/java/io/druid/segment/column/BitmapIndex.java index 573b38982ce..50078bcf8b1 100644 --- a/processing/src/main/java/io/druid/segment/column/BitmapIndex.java +++ b/processing/src/main/java/io/druid/segment/column/BitmapIndex.java @@ -19,15 +19,22 @@ package io.druid.segment.column; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; /** */ public interface BitmapIndex { public int getCardinality(); + public String getValue(int index); + public boolean hasNulls(); - public ImmutableConciseSet getConciseSet(String value); - public ImmutableConciseSet getConciseSet(int idx); + + public BitmapFactory getBitmapFactory(); + + public ImmutableBitmap getBitmap(String value); + + public ImmutableBitmap getBitmap(int idx); } diff --git a/processing/src/main/java/io/druid/segment/data/BitmapCompressedIndexedInts.java b/processing/src/main/java/io/druid/segment/data/BitmapCompressedIndexedInts.java new file mode 100644 index 00000000000..2239f450df1 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/BitmapCompressedIndexedInts.java @@ -0,0 +1,109 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.data; + +import com.google.common.collect.Ordering; +import com.metamx.collections.bitmap.ImmutableBitmap; +import org.roaringbitmap.IntIterator; + +import javax.annotation.Nullable; +import java.util.Iterator; + +/** + */ +public class BitmapCompressedIndexedInts implements IndexedInts, Comparable +{ + private static Ordering comparator = new Ordering() + { + @Override + public int compare( + ImmutableBitmap set, ImmutableBitmap set1 + ) + { + if (set.size() == 0 && set1.size() == 0) { + return 0; + } + if (set.size() == 0) { + return -1; + } + if (set1.size() == 0) { + return 1; + } + return set.compareTo(set1); + } + }.nullsFirst(); + + private final ImmutableBitmap immutableBitmap; + + public BitmapCompressedIndexedInts(ImmutableBitmap immutableBitmap) + { + this.immutableBitmap = immutableBitmap; + } + + @Override + public int compareTo(@Nullable ImmutableBitmap otherBitmap) + { + return comparator.compare(immutableBitmap, otherBitmap); + } + + @Override + public int size() + { + return immutableBitmap.size(); + } + + @Override + public int get(int index) + { + throw new UnsupportedOperationException("This is really slow, so it's just not supported."); + } + + public ImmutableBitmap getImmutableBitmap() + { + return immutableBitmap; + } + + @Override + public Iterator iterator() + { + return new Iterator() + { + IntIterator baseIterator = immutableBitmap.iterator(); + + @Override + public boolean hasNext() + { + return baseIterator.hasNext(); + } + + @Override + public Integer next() + { + return baseIterator.next(); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } +} diff --git a/processing/src/main/java/io/druid/segment/data/BitmapSerde.java b/processing/src/main/java/io/druid/segment/data/BitmapSerde.java new file mode 100644 index 00000000000..6e47dd765e9 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/BitmapSerde.java @@ -0,0 +1,37 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.data; + +import com.metamx.common.ISE; + +public class BitmapSerde +{ + + // default bitmap indices for Druid >= 0.7.x + public static class DefaultBitmapSerdeFactory extends ConciseBitmapSerdeFactory {} + + // default bitmap indices in Druid <= 0.6.x + public static class LegacyBitmapSerdeFactory extends ConciseBitmapSerdeFactory {} + + public static BitmapSerdeFactory createLegacyFactory() + { + return new LegacyBitmapSerdeFactory(); + } +} diff --git a/server/src/main/java/io/druid/metadata/DerbyConnectionFactory.java b/processing/src/main/java/io/druid/segment/data/BitmapSerdeFactory.java similarity index 52% rename from server/src/main/java/io/druid/metadata/DerbyConnectionFactory.java rename to processing/src/main/java/io/druid/segment/data/BitmapSerdeFactory.java index 75f10f6f067..6f7e1f33ce0 100644 --- a/server/src/main/java/io/druid/metadata/DerbyConnectionFactory.java +++ b/processing/src/main/java/io/druid/segment/data/BitmapSerdeFactory.java @@ -17,31 +17,23 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.metadata; +package io.druid.segment.data; -import com.google.api.client.repackaged.com.google.common.base.Throwables; -import org.skife.jdbi.v2.tweak.ConnectionFactory; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; - -public class DerbyConnectionFactory implements ConnectionFactory +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = BitmapSerde.DefaultBitmapSerdeFactory.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "concise", value = ConciseBitmapSerdeFactory.class), + @JsonSubTypes.Type(name = "roaring", value = RoaringBitmapSerdeFactory.class) +}) +public interface BitmapSerdeFactory { - final private String dbName; + public ObjectStrategy getObjectStrategy(); - public DerbyConnectionFactory(String dbName) { - this.dbName = dbName; - } - - public Connection openConnection() throws SQLException { - final String nsURL=String.format("jdbc:derby://localhost:1527/%s;create=true", dbName); - try { - Class.forName("org.apache.derby.jdbc.ClientDriver"); - } catch (Exception e) { - throw Throwables.propagate(e); - } - - return DriverManager.getConnection(nsURL); - } + public BitmapFactory getBitmapFactory(); } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java index 58f57d1191a..f5c1e051f15 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java @@ -28,6 +28,7 @@ import com.metamx.common.IAE; import com.metamx.common.guava.CloseQuietly; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidResourceHolder; +import io.druid.segment.CompressedPools; import java.io.IOException; import java.nio.ByteBuffer; @@ -42,7 +43,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier { public static final byte LZF_VERSION = 0x1; public static final byte version = 0x2; - public static final int MAX_FLOATS_IN_BUFFER = (0xFFFF >> 2); + public static final int MAX_FLOATS_IN_BUFFER = CompressedPools.BUFFER_SIZE / Floats.BYTES; private final int totalSize; private final int sizePer; @@ -70,87 +71,28 @@ public class CompressedFloatsIndexedSupplier implements Supplier @Override public IndexedFloats get() { - return new IndexedFloats() - { - int currIndex = -1; - ResourceHolder holder; - FloatBuffer buffer; + final int div = Integer.numberOfTrailingZeros(sizePer); + final int rem = sizePer - 1; + final boolean powerOf2 = sizePer == (1 << div); + if(powerOf2) { + return new CompressedIndexedFloats() { + @Override + public float get(int index) + { + // optimize division and remainder for powers of 2 + final int bufferNum = index >> div; - @Override - public int size() - { - return totalSize; - } - - @Override - public float get(int index) - { - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - if (bufferNum != currIndex) { - loadBuffer(bufferNum); - } - - return buffer.get(buffer.position() + bufferIndex); - } - - @Override - public void fill(int index, float[] toFill) - { - if (totalSize - index < toFill.length) { - throw new IndexOutOfBoundsException( - String.format( - "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize - ) - ); - } - - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - int leftToFill = toFill.length; - while (leftToFill > 0) { if (bufferNum != currIndex) { loadBuffer(bufferNum); } - buffer.mark(); - buffer.position(buffer.position() + bufferIndex); - final int numToGet = Math.min(buffer.remaining(), leftToFill); - buffer.get(toFill, toFill.length - leftToFill, numToGet); - buffer.reset(); - leftToFill -= numToGet; - ++bufferNum; - bufferIndex = 0; + final int bufferIndex = index & rem; + return buffer.get(buffer.position() + bufferIndex); } - } - - private void loadBuffer(int bufferNum) - { - CloseQuietly.close(holder); - holder = baseFloatBuffers.get(bufferNum); - buffer = holder.get(); - currIndex = bufferNum; - } - - @Override - public String toString() - { - return "CompressedFloatsIndexedSupplier_Anonymous{" + - "currIndex=" + currIndex + - ", sizePer=" + sizePer + - ", numChunks=" + baseFloatBuffers.size() + - ", totalSize=" + totalSize + - '}'; - } - - @Override - public void close() throws IOException - { - Closeables.close(holder, false); - } - }; + }; + } else { + return new CompressedIndexedFloats(); + } } public long getSerializedSize() @@ -185,11 +127,6 @@ public class CompressedFloatsIndexedSupplier implements Supplier return baseFloatBuffers; } - public static int numFloatsInBuffer(int numFloatsInChunk) - { - return MAX_FLOATS_IN_BUFFER - (MAX_FLOATS_IN_BUFFER % numFloatsInChunk); - } - public static CompressedFloatsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); @@ -245,7 +182,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier ) { Preconditions.checkArgument( - chunkFactor * Floats.BYTES <= 0xffff, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor + chunkFactor <= MAX_FLOATS_IN_BUFFER, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor ); return new CompressedFloatsIndexedSupplier( @@ -294,4 +231,85 @@ public class CompressedFloatsIndexedSupplier implements Supplier ); } + private class CompressedIndexedFloats implements IndexedFloats + { + int currIndex = -1; + ResourceHolder holder; + FloatBuffer buffer; + + @Override + public int size() + { + return totalSize; + } + + @Override + public float get(final int index) + { + // division + remainder is optimized by the compiler so keep those together + final int bufferNum = index / sizePer; + final int bufferIndex = index % sizePer; + + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + return buffer.get(buffer.position() + bufferIndex); + } + + @Override + public void fill(int index, float[] toFill) + { + if (totalSize - index < toFill.length) { + throw new IndexOutOfBoundsException( + String.format( + "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize + ) + ); + } + + int bufferNum = index / sizePer; + int bufferIndex = index % sizePer; + + int leftToFill = toFill.length; + while (leftToFill > 0) { + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + buffer.mark(); + buffer.position(buffer.position() + bufferIndex); + final int numToGet = Math.min(buffer.remaining(), leftToFill); + buffer.get(toFill, toFill.length - leftToFill, numToGet); + buffer.reset(); + leftToFill -= numToGet; + ++bufferNum; + bufferIndex = 0; + } + } + + protected void loadBuffer(int bufferNum) + { + CloseQuietly.close(holder); + holder = baseFloatBuffers.get(bufferNum); + buffer = holder.get(); + currIndex = bufferNum; + } + + @Override + public String toString() + { + return "CompressedFloatsIndexedSupplier_Anonymous{" + + "currIndex=" + currIndex + + ", sizePer=" + sizePer + + ", numChunks=" + baseFloatBuffers.size() + + ", totalSize=" + totalSize + + '}'; + } + + @Override + public void close() throws IOException + { + Closeables.close(holder, false); + } + } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java index 578712d16f3..dc3e2149584 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -28,6 +28,7 @@ import com.metamx.common.IAE; import com.metamx.common.guava.CloseQuietly; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidResourceHolder; +import io.druid.segment.CompressedPools; import java.io.IOException; import java.nio.ByteBuffer; @@ -42,6 +43,8 @@ public class CompressedLongsIndexedSupplier implements Supplier { public static final byte LZF_VERSION = 0x1; public static final byte version = 0x2; + public static final int MAX_LONGS_IN_BUFFER = CompressedPools.BUFFER_SIZE / Longs.BYTES; + private final int totalSize; private final int sizePer; @@ -69,99 +72,28 @@ public class CompressedLongsIndexedSupplier implements Supplier @Override public IndexedLongs get() { - return new IndexedLongs() - { - int currIndex = -1; - ResourceHolder holder; - LongBuffer buffer; + final int div = Integer.numberOfTrailingZeros(sizePer); + final int rem = sizePer - 1; + final boolean powerOf2 = sizePer == (1 << div); + if(powerOf2) { + return new CompressedIndexedLongs() { + @Override + public long get(int index) + { + // optimize division and remainder for powers of 2 + final int bufferNum = index >> div; - @Override - public int size() - { - return totalSize; - } - - @Override - public long get(int index) - { - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - if (bufferNum != currIndex) { - loadBuffer(bufferNum); - } - - return buffer.get(buffer.position() + bufferIndex); - } - - @Override - public void fill(int index, long[] toFill) - { - if (totalSize - index < toFill.length) { - throw new IndexOutOfBoundsException( - String.format( - "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize - ) - ); - } - - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - int leftToFill = toFill.length; - while (leftToFill > 0) { if (bufferNum != currIndex) { loadBuffer(bufferNum); } - buffer.mark(); - buffer.position(buffer.position() + bufferIndex); - final int numToGet = Math.min(buffer.remaining(), leftToFill); - buffer.get(toFill, toFill.length - leftToFill, numToGet); - buffer.reset(); - leftToFill -= numToGet; - ++bufferNum; - bufferIndex = 0; + final int bufferIndex = index & rem; + return buffer.get(buffer.position() + bufferIndex); } - } - - private void loadBuffer(int bufferNum) - { - CloseQuietly.close(holder); - holder = baseLongBuffers.get(bufferNum); - buffer = holder.get(); - currIndex = bufferNum; - } - - @Override - public int binarySearch(long key) - { - throw new UnsupportedOperationException(); - } - - @Override - public int binarySearch(long key, int from, int to) - { - throw new UnsupportedOperationException(); - } - - @Override - public String toString() - { - return "CompressedLongsIndexedSupplier_Anonymous{" + - "currIndex=" + currIndex + - ", sizePer=" + sizePer + - ", numChunks=" + baseLongBuffers.size() + - ", totalSize=" + totalSize + - '}'; - } - - @Override - public void close() throws IOException - { - Closeables.close(holder, false); - } - }; + }; + } else { + return new CompressedIndexedLongs(); + } } public long getSerializedSize() @@ -227,7 +159,7 @@ public class CompressedLongsIndexedSupplier implements Supplier public static CompressedLongsIndexedSupplier fromLongBuffer(LongBuffer buffer, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression) { - return fromLongBuffer(buffer, 0xFFFF / Longs.BYTES, byteOrder, compression); + return fromLongBuffer(buffer, MAX_LONGS_IN_BUFFER, byteOrder, compression); } public static CompressedLongsIndexedSupplier fromLongBuffer( @@ -235,7 +167,7 @@ public class CompressedLongsIndexedSupplier implements Supplier ) { Preconditions.checkArgument( - chunkFactor * Longs.BYTES <= 0xffff, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor + chunkFactor <= MAX_LONGS_IN_BUFFER, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor ); return new CompressedLongsIndexedSupplier( @@ -284,4 +216,97 @@ public class CompressedLongsIndexedSupplier implements Supplier ); } + private class CompressedIndexedLongs implements IndexedLongs + { + int currIndex = -1; + ResourceHolder holder; + LongBuffer buffer; + + @Override + public int size() + { + return totalSize; + } + + @Override + public long get(int index) + { + final int bufferNum = index / sizePer; + final int bufferIndex = index % sizePer; + + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + return buffer.get(buffer.position() + bufferIndex); + } + + @Override + public void fill(int index, long[] toFill) + { + if (totalSize - index < toFill.length) { + throw new IndexOutOfBoundsException( + String.format( + "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize + ) + ); + } + + int bufferNum = index / sizePer; + int bufferIndex = index % sizePer; + + int leftToFill = toFill.length; + while (leftToFill > 0) { + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + buffer.mark(); + buffer.position(buffer.position() + bufferIndex); + final int numToGet = Math.min(buffer.remaining(), leftToFill); + buffer.get(toFill, toFill.length - leftToFill, numToGet); + buffer.reset(); + leftToFill -= numToGet; + ++bufferNum; + bufferIndex = 0; + } + } + + protected void loadBuffer(int bufferNum) + { + CloseQuietly.close(holder); + holder = baseLongBuffers.get(bufferNum); + buffer = holder.get(); + currIndex = bufferNum; + } + + @Override + public int binarySearch(long key) + { + throw new UnsupportedOperationException(); + } + + @Override + public int binarySearch(long key, int from, int to) + { + throw new UnsupportedOperationException(); + } + + @Override + public String toString() + { + return "CompressedLongsIndexedSupplier_Anonymous{" + + "currIndex=" + currIndex + + ", sizePer=" + sizePer + + ", numChunks=" + baseLongBuffers.size() + + ", totalSize=" + totalSize + + '}'; + } + + @Override + public void close() throws IOException + { + Closeables.close(holder, false); + } + } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java index cf04cc44767..b1e3c99f614 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java @@ -25,6 +25,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidResourceHolder; +import io.druid.segment.CompressedPools; import java.io.IOException; import java.io.OutputStream; @@ -39,11 +40,10 @@ public class CompressedLongsSupplierSerializer IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression ) throws IOException { - final int sizePer = 0xFFFF / Longs.BYTES; final CompressedLongsSupplierSerializer retVal = new CompressedLongsSupplierSerializer( - sizePer, + CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER, new GenericIndexedWriter>( - ioPeon, filenameBase, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer) + ioPeon, filenameBase, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER) ), compression ); diff --git a/processing/src/main/java/io/druid/segment/data/ConciseBitmapSerdeFactory.java b/processing/src/main/java/io/druid/segment/data/ConciseBitmapSerdeFactory.java new file mode 100644 index 00000000000..5df982ce04d --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/ConciseBitmapSerdeFactory.java @@ -0,0 +1,102 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.data; + +import com.google.common.collect.Ordering; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ConciseBitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; +import com.metamx.collections.bitmap.WrappedImmutableConciseBitmap; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; + +import java.nio.ByteBuffer; + +/** + */ +public class ConciseBitmapSerdeFactory implements BitmapSerdeFactory +{ + private static final ObjectStrategy objectStrategy = new ImmutableConciseSetObjectStrategy(); + private static final BitmapFactory bitmapFactory = new ConciseBitmapFactory(); + + @Override + public ObjectStrategy getObjectStrategy() + { + return objectStrategy; + } + + @Override + public BitmapFactory getBitmapFactory() + { + return bitmapFactory; + } + + private static Ordering conciseComparator = new Ordering() + { + @Override + public int compare( + WrappedImmutableConciseBitmap conciseSet, WrappedImmutableConciseBitmap conciseSet1 + ) + { + if (conciseSet.size() == 0 && conciseSet1.size() == 0) { + return 0; + } + if (conciseSet.size() == 0) { + return -1; + } + if (conciseSet1.size() == 0) { + return 1; + } + return conciseSet.compareTo(conciseSet1); + } + }.nullsFirst(); + + private static class ImmutableConciseSetObjectStrategy + implements ObjectStrategy + { + @Override + public Class getClazz() + { + return ImmutableBitmap.class; + } + + @Override + public WrappedImmutableConciseBitmap fromByteBuffer(ByteBuffer buffer, int numBytes) + { + final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); + readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes); + return new WrappedImmutableConciseBitmap(new ImmutableConciseSet(readOnlyBuffer)); + } + + @Override + public byte[] toBytes(ImmutableBitmap val) + { + if (val == null || val.size() == 0) { + return new byte[]{}; + } + return val.toBytes(); + } + + @Override + public int compare(ImmutableBitmap o1, ImmutableBitmap o2) + { + return conciseComparator.compare((WrappedImmutableConciseBitmap) o1, (WrappedImmutableConciseBitmap) o2); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java b/processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java deleted file mode 100644 index 95b33fc0d9b..00000000000 --- a/processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package io.druid.segment.data; - -import com.google.common.collect.Ordering; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import it.uniroma3.mat.extendedset.intset.IntSet; - -import javax.annotation.Nullable; -import java.nio.ByteBuffer; -import java.util.Iterator; - -/** - */ -public class ConciseCompressedIndexedInts implements IndexedInts, Comparable -{ - public static ObjectStrategy objectStrategy = - new ImmutableConciseSetObjectStrategy(); - - private static Ordering comparator = new Ordering() - { - @Override - public int compare( - @Nullable ImmutableConciseSet conciseSet, @Nullable ImmutableConciseSet conciseSet1 - ) - { - if (conciseSet.size() == 0 && conciseSet1.size() == 0) { - return 0; - } - if (conciseSet.size() == 0) { - return -1; - } - if (conciseSet1.size() == 0) { - return 1; - } - return conciseSet.compareTo(conciseSet1); - } - }.nullsFirst(); - - private final ImmutableConciseSet immutableConciseSet; - - public ConciseCompressedIndexedInts(ImmutableConciseSet conciseSet) - { - this.immutableConciseSet = conciseSet; - } - - @Override - public int compareTo(ConciseCompressedIndexedInts conciseCompressedIndexedInts) - { - return immutableConciseSet.compareTo(conciseCompressedIndexedInts.getImmutableConciseSet()); - } - - @Override - public int size() - { - return immutableConciseSet.size(); - } - - @Override - public int get(int index) - { - throw new UnsupportedOperationException("This is really slow, so it's just not supported."); - } - - public ImmutableConciseSet getImmutableConciseSet() - { - return immutableConciseSet; - } - - @Override - public Iterator iterator() - { - return new Iterator() - { - IntSet.IntIterator baseIterator = immutableConciseSet.iterator(); - - @Override - public boolean hasNext() - { - return baseIterator.hasNext(); - } - - @Override - public Integer next() - { - return baseIterator.next(); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }; - } - - private static class ImmutableConciseSetObjectStrategy - implements ObjectStrategy - { - @Override - public Class getClazz() - { - return ImmutableConciseSet.class; - } - - @Override - public ImmutableConciseSet fromByteBuffer(ByteBuffer buffer, int numBytes) - { - final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); - readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes); - return new ImmutableConciseSet(readOnlyBuffer); - } - - @Override - public byte[] toBytes(ImmutableConciseSet val) - { - if (val == null || val.size() == 0) { - return new byte[]{}; - } - return val.toBytes(); - } - - @Override - public int compare(ImmutableConciseSet o1, ImmutableConciseSet o2) - { - return comparator.compare(o1, o2); - } - } -} diff --git a/processing/src/main/java/io/druid/segment/data/IndexedRTree.java b/processing/src/main/java/io/druid/segment/data/IndexedRTree.java index d0785a21a12..f3a3d8b3ada 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedRTree.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedRTree.java @@ -20,6 +20,7 @@ package io.druid.segment.data; import com.google.common.collect.Ordering; +import com.metamx.collections.bitmap.BitmapFactory; import com.metamx.collections.spatial.ImmutableRTree; import java.nio.ByteBuffer; @@ -28,9 +29,6 @@ import java.nio.ByteBuffer; */ public class IndexedRTree implements Comparable { - public static ObjectStrategy objectStrategy = - new ImmutableRTreeObjectStrategy(); - private static Ordering comparator = new Ordering() { @Override @@ -69,9 +67,16 @@ public class IndexedRTree implements Comparable return immutableRTree; } - private static class ImmutableRTreeObjectStrategy + public static class ImmutableRTreeObjectStrategy implements ObjectStrategy { + private final BitmapFactory bitmapFactory; + + public ImmutableRTreeObjectStrategy(BitmapFactory bitmapFactory) + { + this.bitmapFactory = bitmapFactory; + } + @Override public Class getClazz() { @@ -81,10 +86,9 @@ public class IndexedRTree implements Comparable @Override public ImmutableRTree fromByteBuffer(ByteBuffer buffer, int numBytes) { - final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes); - return new ImmutableRTree(readOnlyBuffer); + return new ImmutableRTree(readOnlyBuffer, bitmapFactory); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java new file mode 100644 index 00000000000..27cb7a9399a --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java @@ -0,0 +1,103 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.data; + +import com.google.common.collect.Ordering; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; +import com.metamx.collections.bitmap.RoaringBitmapFactory; +import com.metamx.collections.bitmap.WrappedImmutableRoaringBitmap; +import org.roaringbitmap.buffer.ImmutableRoaringBitmap; + +import java.nio.ByteBuffer; + +/** + */ +public class RoaringBitmapSerdeFactory implements BitmapSerdeFactory +{ + private static final ObjectStrategy objectStrategy = new ImmutableRoaringBitmapObjectStrategy(); + private static final BitmapFactory bitmapFactory = new RoaringBitmapFactory(); + + @Override + public ObjectStrategy getObjectStrategy() + { + return objectStrategy; + } + + @Override + public BitmapFactory getBitmapFactory() + { + return bitmapFactory; + } + + private static Ordering roaringComparator = new Ordering() + { + @Override + public int compare( + WrappedImmutableRoaringBitmap set1, WrappedImmutableRoaringBitmap set2 + ) + { + if (set1.size() == 0 && set2.size() == 0) { + return 0; + } + if (set1.size() == 0) { + return -1; + } + if (set2.size() == 0) { + return 1; + } + + return set1.compareTo(set2); + } + }.nullsFirst(); + + private static class ImmutableRoaringBitmapObjectStrategy + implements ObjectStrategy + { + @Override + public Class getClazz() + { + return ImmutableBitmap.class; + } + + @Override + public ImmutableBitmap fromByteBuffer(ByteBuffer buffer, int numBytes) + { + final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); + readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes); + return new WrappedImmutableRoaringBitmap(new ImmutableRoaringBitmap(readOnlyBuffer)); + } + + @Override + public byte[] toBytes(ImmutableBitmap val) + { + if (val == null || val.size() == 0) { + return new byte[]{}; + } + return val.toBytes(); + } + + @Override + public int compare(ImmutableBitmap o1, ImmutableBitmap o2) + { + return roaringComparator.compare((WrappedImmutableRoaringBitmap) o1, (WrappedImmutableRoaringBitmap) o2); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/filter/AndFilter.java b/processing/src/main/java/io/druid/segment/filter/AndFilter.java index 70b374459db..58e20579c30 100644 --- a/processing/src/main/java/io/druid/segment/filter/AndFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/AndFilter.java @@ -20,11 +20,11 @@ package io.druid.segment.filter; import com.google.common.collect.Lists; +import com.metamx.collections.bitmap.ImmutableBitmap; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; @@ -42,18 +42,18 @@ public class AndFilter implements Filter } @Override - public ImmutableConciseSet goConcise(BitmapIndexSelector selector) + public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector) { if (filters.size() == 1) { - return filters.get(0).goConcise(selector); + return filters.get(0).getBitmapIndex(selector); } - List conciseSets = Lists.newArrayList(); + List bitmaps = Lists.newArrayList(); for (int i = 0; i < filters.size(); i++) { - conciseSets.add(filters.get(i).goConcise(selector)); + bitmaps.add(filters.get(i).getBitmapIndex(selector)); } - return ImmutableConciseSet.intersection(conciseSets); + return selector.getBitmapFactory().intersection(bitmaps); } @Override diff --git a/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java b/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java index b3f06b949a3..44cd124cc33 100644 --- a/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java @@ -21,13 +21,13 @@ package io.druid.segment.filter; import com.google.common.base.Function; import com.google.common.base.Predicate; +import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.common.guava.FunctionalIterable; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; import io.druid.segment.data.Indexed; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import javax.annotation.Nullable; @@ -48,23 +48,23 @@ class DimensionPredicateFilter implements Filter } @Override - public ImmutableConciseSet goConcise(final BitmapIndexSelector selector) + public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector) { Indexed dimValues = selector.getDimensionValues(dimension); if (dimValues == null || dimValues.size() == 0 || predicate == null) { - return new ImmutableConciseSet(); + return selector.getBitmapFactory().makeEmptyImmutableBitmap(); } - return ImmutableConciseSet.union( + return selector.getBitmapFactory().union( FunctionalIterable.create(dimValues) .filter(predicate) .transform( - new Function() + new Function() { @Override - public ImmutableConciseSet apply(@Nullable String input) + public ImmutableBitmap apply(@Nullable String input) { - return selector.getConciseInvertedIndex(dimension, input); + return selector.getBitmapIndex(dimension, input); } } ) diff --git a/processing/src/main/java/io/druid/segment/filter/ExtractionFilter.java b/processing/src/main/java/io/druid/segment/filter/ExtractionFilter.java index 62a92ca193a..64b4a2e3bdc 100644 --- a/processing/src/main/java/io/druid/segment/filter/ExtractionFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/ExtractionFilter.java @@ -20,13 +20,13 @@ package io.druid.segment.filter; import com.google.common.collect.Lists; +import com.metamx.collections.bitmap.ImmutableBitmap; import io.druid.query.extraction.DimExtractionFn; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; import io.druid.segment.data.Indexed; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; @@ -34,8 +34,6 @@ import java.util.List; */ public class ExtractionFilter implements Filter { - private static final int MAX_SIZE = 50000; - private final String dimension; private final String value; private final DimExtractionFn fn; @@ -67,9 +65,9 @@ public class ExtractionFilter implements Filter } @Override - public ImmutableConciseSet goConcise(BitmapIndexSelector selector) + public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector) { - return new OrFilter(makeFilters(selector)).goConcise(selector); + return new OrFilter(makeFilters(selector)).getBitmapIndex(selector); } @Override diff --git a/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java b/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java index 75817403571..79f13c22808 100644 --- a/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java @@ -21,13 +21,13 @@ package io.druid.segment.filter; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.common.guava.FunctionalIterable; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; import io.druid.segment.data.Indexed; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.mozilla.javascript.Context; import org.mozilla.javascript.Function; import org.mozilla.javascript.ScriptableObject; @@ -46,39 +46,42 @@ public class JavaScriptFilter implements Filter } @Override - public ImmutableConciseSet goConcise(final BitmapIndexSelector selector) + public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector) { final Context cx = Context.enter(); try { final Indexed dimValues = selector.getDimensionValues(dimension); - ImmutableConciseSet conciseSet; + ImmutableBitmap bitmap; if (dimValues == null) { - conciseSet = new ImmutableConciseSet(); + bitmap = selector.getBitmapFactory().makeEmptyImmutableBitmap(); } else { - conciseSet = ImmutableConciseSet.union( + bitmap = selector.getBitmapFactory().union( FunctionalIterable.create(dimValues) - .filter(new Predicate() - { - @Override - public boolean apply(@Nullable String input) - { - return predicate.applyInContext(cx, input); - } - }) - .transform( - new com.google.common.base.Function() - { - @Override - public ImmutableConciseSet apply(@Nullable String input) - { - return selector.getConciseInvertedIndex(dimension, input); - } - } - ) + .filter( + new Predicate() + { + @Override + public boolean apply(@Nullable String input) + { + return predicate.applyInContext(cx, input); + } + } + ) + .transform( + new com.google.common.base.Function() + { + @Override + public ImmutableBitmap apply(@Nullable String input) + { + return selector.getBitmapIndex(dimension, input); + } + } + ) ); } - return conciseSet; - } finally { + return bitmap; + } + finally { Context.exit(); } } @@ -107,7 +110,8 @@ public class JavaScriptFilter implements Filter scope = cx.initStandardObjects(); fnApply = cx.compileFunction(scope, script, "script", 1, null); - } finally { + } + finally { Context.exit(); } } @@ -119,7 +123,8 @@ public class JavaScriptFilter implements Filter final Context cx = Context.enter(); try { return applyInContext(cx, input); - } finally { + } + finally { Context.exit(); } diff --git a/processing/src/main/java/io/druid/segment/filter/NotFilter.java b/processing/src/main/java/io/druid/segment/filter/NotFilter.java index d517613f978..f5726657ead 100644 --- a/processing/src/main/java/io/druid/segment/filter/NotFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/NotFilter.java @@ -19,11 +19,11 @@ package io.druid.segment.filter; +import com.metamx.collections.bitmap.ImmutableBitmap; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** */ @@ -39,10 +39,10 @@ public class NotFilter implements Filter } @Override - public ImmutableConciseSet goConcise(BitmapIndexSelector selector) + public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector) { - return ImmutableConciseSet.complement( - baseFilter.goConcise(selector), + return selector.getBitmapFactory().complement( + baseFilter.getBitmapIndex(selector), selector.getNumRows() ); } diff --git a/processing/src/main/java/io/druid/segment/filter/OrFilter.java b/processing/src/main/java/io/druid/segment/filter/OrFilter.java index 8476066a933..42ffde83928 100644 --- a/processing/src/main/java/io/druid/segment/filter/OrFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/OrFilter.java @@ -20,11 +20,11 @@ package io.druid.segment.filter; import com.google.common.collect.Lists; +import com.metamx.collections.bitmap.ImmutableBitmap; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; @@ -46,18 +46,18 @@ public class OrFilter implements Filter } @Override - public ImmutableConciseSet goConcise(BitmapIndexSelector selector) + public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector) { if (filters.size() == 1) { - return filters.get(0).goConcise(selector); + return filters.get(0).getBitmapIndex(selector); } - List conciseSets = Lists.newArrayList(); + List bitmaps = Lists.newArrayList(); for (int i = 0; i < filters.size(); i++) { - conciseSets.add(filters.get(i).goConcise(selector)); + bitmaps.add(filters.get(i).getBitmapIndex(selector)); } - return ImmutableConciseSet.union(conciseSets); + return selector.getBitmapFactory().union(bitmaps); } @Override diff --git a/processing/src/main/java/io/druid/segment/filter/SelectorFilter.java b/processing/src/main/java/io/druid/segment/filter/SelectorFilter.java index f57d040a1c0..b16dd567a90 100644 --- a/processing/src/main/java/io/druid/segment/filter/SelectorFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/SelectorFilter.java @@ -19,11 +19,11 @@ package io.druid.segment.filter; +import com.metamx.collections.bitmap.ImmutableBitmap; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** */ @@ -42,9 +42,9 @@ public class SelectorFilter implements Filter } @Override - public ImmutableConciseSet goConcise(BitmapIndexSelector selector) + public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector) { - return selector.getConciseInvertedIndex(dimension, value); + return selector.getBitmapIndex(dimension, value); } @Override diff --git a/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java b/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java index 538b133be57..c2f77690fd4 100644 --- a/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java @@ -18,12 +18,12 @@ */ package io.druid.segment.filter; +import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.spatial.search.Bound; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** */ @@ -42,9 +42,13 @@ public class SpatialFilter implements Filter } @Override - public ImmutableConciseSet goConcise(final BitmapIndexSelector selector) + public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector) { - return ImmutableConciseSet.union(selector.getSpatialIndex(dimension).search(bound)); + Iterable search = selector.getSpatialIndex(dimension).search(bound); + for (ImmutableBitmap immutableBitmap : search) { + System.out.println(immutableBitmap); + } + return selector.getBitmapFactory().union(search); } @Override diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java index 1addc0551e1..f37a9962fb3 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -21,6 +21,8 @@ package io.druid.segment.incremental; import com.google.common.base.Function; import com.google.common.collect.Maps; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.MutableBitmap; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; import io.druid.segment.IndexableAdapter; @@ -31,9 +33,8 @@ import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.ListIndexed; -import it.uniroma3.mat.extendedset.intset.ConciseSet; -import it.uniroma3.mat.extendedset.intset.IntSet; import org.joda.time.Interval; +import org.roaringbitmap.IntIterator; import javax.annotation.Nullable; import java.util.Iterator; @@ -46,10 +47,10 @@ public class IncrementalIndexAdapter implements IndexableAdapter private static final Logger log = new Logger(IncrementalIndexAdapter.class); private final Interval dataInterval; private final IncrementalIndex index; - private final Map> invertedIndexes; + private final Map> invertedIndexes; public IncrementalIndexAdapter( - Interval dataInterval, IncrementalIndex index + Interval dataInterval, IncrementalIndex index, BitmapFactory bitmapFactory ) { this.dataInterval = dataInterval; @@ -58,7 +59,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter this.invertedIndexes = Maps.newHashMap(); for (String dimension : index.getDimensions()) { - invertedIndexes.put(dimension, Maps.newHashMap()); + invertedIndexes.put(dimension, Maps.newHashMap()); } int rowNum = 0; @@ -67,10 +68,10 @@ public class IncrementalIndexAdapter implements IndexableAdapter for (String dimension : index.getDimensions()) { int dimIndex = index.getDimensionIndex(dimension); - Map conciseSets = invertedIndexes.get(dimension); + Map bitmapIndexes = invertedIndexes.get(dimension); - if (conciseSets == null || dims == null) { - log.error("conciseSets and dims are null!"); + if (bitmapIndexes == null || dims == null) { + log.error("bitmapIndexes and dims are null!"); continue; } if (dimIndex >= dims.length || dims[dimIndex] == null) { @@ -78,15 +79,15 @@ public class IncrementalIndexAdapter implements IndexableAdapter } for (String dimValue : dims[dimIndex]) { - ConciseSet conciseSet = conciseSets.get(dimValue); + MutableBitmap mutableBitmap = bitmapIndexes.get(dimValue); - if (conciseSet == null) { - conciseSet = new ConciseSet(); - conciseSets.put(dimValue, conciseSet); + if (mutableBitmap == null) { + mutableBitmap = bitmapFactory.makeEmptyMutableBitmap(); + bitmapIndexes.put(dimValue, mutableBitmap); } try { - conciseSet.add(rowNum); + mutableBitmap.add(rowNum); } catch (Exception e) { log.info(e.toString()); @@ -220,17 +221,17 @@ public class IncrementalIndexAdapter implements IndexableAdapter } @Override - public IndexedInts getInverteds(String dimension, String value) + public IndexedInts getBitmapIndex(String dimension, String value) { - Map dimInverted = invertedIndexes.get(dimension); + Map dimInverted = invertedIndexes.get(dimension); if (dimInverted == null) { return new EmptyIndexedInts(); } - final ConciseSet conciseSet = dimInverted.get(value); + final MutableBitmap bitmapIndex = dimInverted.get(value); - if (conciseSet == null) { + if (bitmapIndex == null) { return new EmptyIndexedInts(); } @@ -239,7 +240,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter @Override public int size() { - return conciseSet.size(); + return bitmapIndex.size(); } @Override @@ -253,7 +254,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter { return new Iterator() { - IntSet.IntIterator baseIter = conciseSet.iterator(); + IntIterator baseIter = bitmapIndex.iterator(); @Override public boolean hasNext() diff --git a/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java b/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java index d36e6a684e1..e1265b0c1ef 100644 --- a/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java @@ -20,23 +20,26 @@ package io.druid.segment.serde; import com.google.common.base.Supplier; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; import io.druid.segment.column.BitmapIndex; import io.druid.segment.data.GenericIndexed; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** -*/ + */ public class BitmapIndexColumnPartSupplier implements Supplier { - private static final ImmutableConciseSet EMPTY_SET = new ImmutableConciseSet(); - - private final GenericIndexed bitmaps; + private final BitmapFactory bitmapFactory; + private final GenericIndexed bitmaps; private final GenericIndexed dictionary; public BitmapIndexColumnPartSupplier( - GenericIndexed bitmaps, + BitmapFactory bitmapFactory, + GenericIndexed bitmaps, GenericIndexed dictionary - ) { + ) + { + this.bitmapFactory = bitmapFactory; this.bitmaps = bitmaps; this.dictionary = dictionary; } @@ -65,22 +68,28 @@ public class BitmapIndexColumnPartSupplier implements Supplier } @Override - public ImmutableConciseSet getConciseSet(String value) + public BitmapFactory getBitmapFactory() { - final int index = dictionary.indexOf(value); - - return getConciseSet(index); + return bitmapFactory; } @Override - public ImmutableConciseSet getConciseSet(int idx) + public ImmutableBitmap getBitmap(String value) + { + final int index = dictionary.indexOf(value); + + return getBitmap(index); + } + + @Override + public ImmutableBitmap getBitmap(int idx) { if (idx < 0) { - return EMPTY_SET; + return bitmapFactory.makeEmptyImmutableBitmap(); } - final ImmutableConciseSet bitmap = bitmaps.get(idx); - return bitmap == null ? EMPTY_SET : bitmap; + final ImmutableBitmap bitmap = bitmaps.get(idx); + return bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap; } }; } diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index c632085a5b0..6eb7011fb8b 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -22,18 +22,19 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Ints; +import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.IAE; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; +import io.druid.segment.data.BitmapSerde; +import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferSerializer; -import io.druid.segment.data.ConciseCompressedIndexedInts; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.IndexedRTree; import io.druid.segment.data.VSizeIndexed; import io.druid.segment.data.VSizeIndexedInts; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.io.IOException; import java.nio.ByteBuffer; @@ -44,11 +45,12 @@ import java.nio.channels.WritableByteChannel; public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde { private final boolean isSingleValued; + private final BitmapSerdeFactory bitmapSerdeFactory; private final GenericIndexed dictionary; private final VSizeIndexedInts singleValuedColumn; private final VSizeIndexed multiValuedColumn; - private final GenericIndexed bitmaps; + private final GenericIndexed bitmaps; private final ImmutableRTree spatialIndex; private final long size; @@ -57,11 +59,14 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde GenericIndexed dictionary, VSizeIndexedInts singleValCol, VSizeIndexed multiValCol, - GenericIndexed bitmaps, + BitmapSerdeFactory bitmapSerdeFactory, + GenericIndexed bitmaps, ImmutableRTree spatialIndex ) { this.isSingleValued = multiValCol == null; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.dictionary = dictionary; this.singleValuedColumn = singleValCol; this.multiValuedColumn = multiValCol; @@ -86,10 +91,14 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde @JsonCreator public DictionaryEncodedColumnPartSerde( - @JsonProperty("isSingleValued") boolean isSingleValued + @JsonProperty("isSingleValued") boolean isSingleValued, + @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory ) { this.isSingleValued = isSingleValued; + this.bitmapSerdeFactory = bitmapSerdeFactory == null + ? new BitmapSerde.LegacyBitmapSerdeFactory() + : bitmapSerdeFactory; this.dictionary = null; this.singleValuedColumn = null; @@ -105,6 +114,12 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde return isSingleValued; } + @JsonProperty + public BitmapSerdeFactory getBitmapSerdeFactory() + { + return bitmapSerdeFactory; + } + @Override public long numBytes() { @@ -135,7 +150,11 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde } if (spatialIndex != null) { - ByteBufferSerializer.writeToChannel(spatialIndex, IndexedRTree.objectStrategy, channel); + ByteBufferSerializer.writeToChannel( + spatialIndex, + new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()), + channel + ); } } @@ -153,23 +172,43 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde singleValuedColumn = VSizeIndexedInts.readFromByteBuffer(buffer); multiValuedColumn = null; builder.setHasMultipleValues(false) - .setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, singleValuedColumn, null, columnConfig.columnCacheSizeBytes())); + .setDictionaryEncodedColumn( + new DictionaryEncodedColumnSupplier( + dictionary, + singleValuedColumn, + null, + columnConfig.columnCacheSizeBytes() + ) + ); } else { singleValuedColumn = null; multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer); builder.setHasMultipleValues(true) - .setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, null, multiValuedColumn, columnConfig.columnCacheSizeBytes())); + .setDictionaryEncodedColumn( + new DictionaryEncodedColumnSupplier( + dictionary, + null, + multiValuedColumn, + columnConfig.columnCacheSizeBytes() + ) + ); } - GenericIndexed bitmaps = GenericIndexed.read( - buffer, ConciseCompressedIndexedInts.objectStrategy + GenericIndexed bitmaps = GenericIndexed.read( + buffer, bitmapSerdeFactory.getObjectStrategy() + ); + builder.setBitmapIndex( + new BitmapIndexColumnPartSupplier( + bitmapSerdeFactory.getBitmapFactory(), + bitmaps, + dictionary + ) ); - builder.setBitmapIndex(new BitmapIndexColumnPartSupplier(bitmaps, dictionary)); ImmutableRTree spatialIndex = null; if (buffer.hasRemaining()) { spatialIndex = ByteBufferSerializer.read( - buffer, IndexedRTree.objectStrategy + buffer, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()) ); builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(spatialIndex)); } @@ -178,6 +217,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde dictionary, singleValuedColumn, multiValuedColumn, + bitmapSerdeFactory, bitmaps, spatialIndex ); diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 3adfed65c06..f79e9279026 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -9,6 +9,7 @@ import com.metamx.common.guava.Sequences; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.segment.SegmentMissingException; import org.joda.time.DateTime; @@ -63,14 +64,21 @@ public class RetryQueryRunnerTest return Sequences.empty(); } }, + (QueryToolChest) new TimeseriesQueryQueryToolChest( + new QueryConfig() + ), new RetryQueryRunnerConfig() { - private int numTries = 0; - private boolean returnPartialResults = true; + @Override + public int getNumTries() { + return 0; + } - public int numTries() { return numTries; } - - public boolean returnPartialResults() { return returnPartialResults; } + @Override + public boolean isReturnPartialResults() + { + return true; + } }, jsonMapper ); @@ -128,12 +136,15 @@ public class RetryQueryRunnerTest } } }, + (QueryToolChest) new TimeseriesQueryQueryToolChest( + new QueryConfig() + ), new RetryQueryRunnerConfig() { private int numTries = 1; private boolean returnPartialResults = true; - public int numTries() { return numTries; } + public int getNumTries() { return numTries; } public boolean returnPartialResults() { return returnPartialResults; } }, @@ -192,12 +203,15 @@ public class RetryQueryRunnerTest } } }, + (QueryToolChest) new TimeseriesQueryQueryToolChest( + new QueryConfig() + ), new RetryQueryRunnerConfig() { private int numTries = 4; private boolean returnPartialResults = true; - public int numTries() { return numTries; } + public int getNumTries() { return numTries; } public boolean returnPartialResults() { return returnPartialResults; } }, @@ -241,12 +255,15 @@ public class RetryQueryRunnerTest return Sequences.empty(); } }, + (QueryToolChest) new TimeseriesQueryQueryToolChest( + new QueryConfig() + ), new RetryQueryRunnerConfig() { private int numTries = 1; private boolean returnPartialResults = false; - public int numTries() { return numTries; } + public int getNumTries() { return numTries; } public boolean returnPartialResults() { return returnPartialResults; } }, diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 2babeadb78a..f2075502e5d 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -1024,6 +1024,48 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); } + @Test + public void testGroupByWithOrderLimit4() + { + GroupByQuery query = new GroupByQuery.Builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setGranularity(QueryRunnerTestHelper.allGran) + .setDimensions( + Arrays.asList( + new DefaultDimensionSpec( + QueryRunnerTestHelper.marketDimension, + QueryRunnerTestHelper.marketDimension + ) + ) + ) + .setInterval(QueryRunnerTestHelper.fullOnInterval) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + QueryRunnerTestHelper.marketDimension, + OrderByColumnSpec.Direction.DESCENDING + ) + ), 3 + ) + ) + .setAggregatorSpecs( + Lists.newArrayList( + QueryRunnerTestHelper.rowsCount + ) + ) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "market", "upfront", "rows", 186L), + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "market", "total_market", "rows", 186L), + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "market", "spot", "rows", 837L) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); + } + @Test public void testHavingSpec() { diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index 56842f16ec2..63fde529c21 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -88,12 +88,35 @@ public class SearchQueryRunnerTest QueryRunnerTestHelper.qualityDimension, Sets.newHashSet("automotive", "mezzanine", "travel", "health", "entertainment") ); - expectedResults.put(QueryRunnerTestHelper.marketDimension.toLowerCase(), Sets.newHashSet("total_market")); + expectedResults.put(QueryRunnerTestHelper.marketDimension, Sets.newHashSet("total_market")); expectedResults.put(QueryRunnerTestHelper.placementishDimension, Sets.newHashSet("a")); checkSearchQuery(searchQuery, expectedResults); } + @Test + public void testSearchSameValueInMultiDims() + { + SearchQuery searchQuery = Druids.newSearchQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .dimensions( + Arrays.asList( + QueryRunnerTestHelper.placementDimension, + QueryRunnerTestHelper.placementishDimension + ) + ) + .query("e") + .build(); + + Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + expectedResults.put(QueryRunnerTestHelper.placementDimension, Sets.newHashSet("preferred")); + expectedResults.put(QueryRunnerTestHelper.placementishDimension, Sets.newHashSet("e", "preferred")); + + checkSearchQuery(searchQuery, expectedResults); + } + @Test public void testFragmentSearch() { diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java new file mode 100644 index 00000000000..01127e06876 --- /dev/null +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -0,0 +1,212 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.spec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.guava.Accumulator; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.common.guava.Yielder; +import com.metamx.common.guava.YieldingAccumulator; +import io.druid.granularity.QueryGranularity; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Druids; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.Result; +import io.druid.query.RetryQueryRunner; +import io.druid.query.SegmentDescriptor; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregator; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultBuilder; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.SegmentMissingException; +import junit.framework.Assert; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class SpecificSegmentQueryRunnerTest +{ + @Test + public void testRetry() throws Exception + { + final ObjectMapper mapper = new DefaultObjectMapper(); + SegmentDescriptor descriptor = new SegmentDescriptor( + new Interval("2012-01-01T00:00:00Z/P1D"), + "version", + 0 + ); + + final SpecificSegmentQueryRunner queryRunner = new SpecificSegmentQueryRunner( + new QueryRunner() + { + @Override + public Sequence run(Query query, Map context) + { + return new Sequence() + { + @Override + public Object accumulate(Object initValue, Accumulator accumulator) + { + throw new SegmentMissingException("FAILSAUCE"); + } + + @Override + public Yielder toYielder( + Object initValue, YieldingAccumulator accumulator + ) + { + return null; + } + }; + + } + }, + new SpecificSegmentSpec( + descriptor + ) + ); + + final Map responseContext = Maps.newHashMap(); + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("foo") + .granularity(QueryGranularity.ALL) + .intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D"))) + .aggregators( + ImmutableList.of( + new CountAggregatorFactory("rows") + ) + ) + .build(); + Sequence results = queryRunner.run( + query, + responseContext + ); + Sequences.toList(results, Lists.newArrayList()); + + Object missingSegments = responseContext.get(RetryQueryRunner.MISSING_SEGMENTS_KEY); + + Assert.assertTrue(missingSegments != null); + Assert.assertTrue(missingSegments instanceof List); + + Object segmentDesc = ((List) missingSegments).get(0); + + Assert.assertTrue(segmentDesc instanceof SegmentDescriptor); + + SegmentDescriptor newDesc = mapper.readValue(mapper.writeValueAsString(segmentDesc), SegmentDescriptor.class); + + Assert.assertEquals(descriptor, newDesc); + } + + @SuppressWarnings("unchecked") + @Test + public void testRetry2() throws Exception + { + final ObjectMapper mapper = new DefaultObjectMapper(); + SegmentDescriptor descriptor = new SegmentDescriptor( + new Interval("2012-01-01T00:00:00Z/P1D"), + "version", + 0 + ); + + TimeseriesResultBuilder builder = new TimeseriesResultBuilder( + new DateTime("2012-01-01T00:00:00Z") + ); + CountAggregator rows = new CountAggregator("rows"); + rows.aggregate(); + builder.addMetric(rows); + final Result value = builder.build(); + + final SpecificSegmentQueryRunner queryRunner = new SpecificSegmentQueryRunner( + new QueryRunner() + { + @Override + public Sequence run(Query query, Map context) + { + return Sequences.withEffect( + Sequences.simple(Arrays.asList(value)), + new Runnable() + { + @Override + public void run() + { + throw new SegmentMissingException("FAILSAUCE"); + } + }, + MoreExecutors.sameThreadExecutor() + ); + } + }, + new SpecificSegmentSpec( + descriptor + ) + ); + + final Map responseContext = Maps.newHashMap(); + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("foo") + .granularity(QueryGranularity.ALL) + .intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D"))) + .aggregators( + ImmutableList.of( + new CountAggregatorFactory("rows") + ) + ) + .build(); + Sequence results = queryRunner.run( + query, + responseContext + ); + List> res = Sequences.toList( + results, + Lists.>newArrayList() + ); + + Assert.assertEquals(1, res.size()); + + Result theVal = res.get(0); + + Assert.assertTrue(1L == theVal.getValue().getLongMetric("rows")); + + Object missingSegments = responseContext.get(RetryQueryRunner.MISSING_SEGMENTS_KEY); + + Assert.assertTrue(missingSegments != null); + Assert.assertTrue(missingSegments instanceof List); + + Object segmentDesc = ((List) missingSegments).get(0); + + Assert.assertTrue(segmentDesc instanceof SegmentDescriptor); + + SegmentDescriptor newDesc = mapper.readValue(mapper.writeValueAsString(segmentDesc), SegmentDescriptor.class); + + Assert.assertEquals(descriptor, newDesc); + } +} \ No newline at end of file diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java new file mode 100644 index 00000000000..b99fa664434 --- /dev/null +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java @@ -0,0 +1,167 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.carrotsearch.junitbenchmarks.AbstractBenchmark; +import com.carrotsearch.junitbenchmarks.BenchmarkOptions; +import com.google.common.base.Supplier; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.druid.collections.StupidPool; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.segment.IncrementalIndexSegment; +import io.druid.segment.QueryableIndexSegment; +import io.druid.segment.TestIndex; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Based on TopNQueryRunnerTest + */ +public class TopNQueryRunnerBenchmark extends AbstractBenchmark +{ + + public static enum TestCases + { + rtIndex, mMappedTestIndex, mergedRealtimeIndex, rtIndexOffheap + } + + private static final String marketDimension = "market"; + private static final String segmentId = "testSegment"; + + private static final HashMap context = new HashMap(); + + private static final TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(marketDimension) + .metric(QueryRunnerTestHelper.indexMetric) + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + private static final Map testCaseMap = Maps.newHashMap(); + + @BeforeClass + public static void setUp() throws Exception + { + QueryRunnerFactory factory = new TopNQueryRunnerFactory( + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + // See OffheapByteBufferPool + // Instead of causing a circular dependency, we simply mimic its behavior + return ByteBuffer.allocateDirect(2000); + } + } + ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + testCaseMap.put( + TestCases.rtIndex, + QueryRunnerTestHelper.makeQueryRunner( + factory, + new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(false), segmentId) + ) + ); + testCaseMap.put( + TestCases.mMappedTestIndex, + QueryRunnerTestHelper.makeQueryRunner( + factory, + new QueryableIndexSegment(segmentId, TestIndex.getMMappedTestIndex()) + ) + ); + testCaseMap.put( + TestCases.mergedRealtimeIndex, + QueryRunnerTestHelper.makeQueryRunner( + factory, + new QueryableIndexSegment(segmentId, TestIndex.mergedRealtimeIndex()) + ) + ); + testCaseMap.put( + TestCases.rtIndexOffheap, + QueryRunnerTestHelper.makeQueryRunner( + factory, + new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(true), segmentId) + ) + ); + //Thread.sleep(10000); + } + + @BenchmarkOptions(warmupRounds = 10000, benchmarkRounds = 10000) + @Test + public void testmMapped() + { + testCaseMap.get(TestCases.mMappedTestIndex).run(query, context); + } + + @Ignore + @BenchmarkOptions(warmupRounds = 10000, benchmarkRounds = 10000) + @Test + public void testrtIndex() + { + testCaseMap.get(TestCases.rtIndex).run(query, context); + } + + @Ignore + @BenchmarkOptions(warmupRounds = 10000, benchmarkRounds = 10000) + @Test + public void testMerged() + { + testCaseMap.get(TestCases.mergedRealtimeIndex).run(query, context); + } + + @Ignore + @BenchmarkOptions(warmupRounds = 10000, benchmarkRounds = 10000) + @Test + public void testOffHeap() + { + testCaseMap.get(TestCases.rtIndexOffheap).run(query, context); + } +} diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 2e978dd408f..acc1a7bb382 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -19,10 +19,12 @@ package io.druid.query.topn; +import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; import io.druid.query.BySegmentResultValueClass; @@ -47,6 +49,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -167,7 +170,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -231,7 +234,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -296,11 +299,89 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } + @Test + public void testTopNBySegment() + { + + final HashMap specialContext = new HashMap(); + specialContext.put("bySegment", "true"); + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(marketDimension) + .metric(QueryRunnerTestHelper.indexMetric) + .threshold(4) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .context(specialContext) + .build(); + + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "addRowsIndexConstant", 5356.814697265625D, + "index", 5351.814697265625D, + marketDimension, "total_market", + "uniques", QueryRunnerTestHelper.UNIQUES_2, + "rows", 4L + ), + ImmutableMap.of( + "addRowsIndexConstant", 4880.669677734375D, + "index", 4875.669677734375D, + marketDimension, "upfront", + "uniques", QueryRunnerTestHelper.UNIQUES_2, + "rows", 4L + ), + ImmutableMap.of( + "addRowsIndexConstant", 2250.8768157958984D, + "index", 2231.8768157958984D, + marketDimension, "spot", + "uniques", QueryRunnerTestHelper.UNIQUES_9, + "rows", 18L + ) + ) + ) + ) + ); + Sequence> results = new TopNQueryQueryToolChest(new TopNQueryConfig()).postMergeQueryDecoration( + runner + ).run( + query, + specialContext + ); + List> resultList = Sequences.toList( + Sequences.map( + results, new Function, Result>() + { + + @Nullable + @Override + public Result apply( + Result input + ) + { + return new Result( + input.getTimestamp(), + (BySegmentTopNResultValue) input.getValue() + ); + } + } + ), + Lists.>newArrayList() + ); + TestHelper.assertExpectedResults(expectedResults, resultList.get(0).getValue().getResults()); + } + @Test public void testTopN() { @@ -346,7 +427,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -395,7 +476,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -444,7 +525,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -486,7 +567,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -521,7 +602,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -570,7 +651,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -623,7 +704,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -665,7 +746,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -683,7 +764,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Lists.>newArrayList( new Result( @@ -722,7 +803,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Lists.>newArrayList( new Result( @@ -748,7 +829,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Sequences.toList( runner.run( @@ -783,7 +864,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Sequences.toList( runner.run( @@ -843,7 +924,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -892,7 +973,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -948,7 +1029,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -996,7 +1077,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1037,7 +1118,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1078,7 +1159,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1119,7 +1200,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1160,7 +1241,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1212,7 +1293,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1264,7 +1345,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1316,7 +1397,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1361,7 +1442,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1407,7 +1488,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1452,7 +1533,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1501,7 +1582,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1586,7 +1667,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1669,7 +1750,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } } diff --git a/processing/src/test/java/io/druid/segment/ConciseOffsetTest.java b/processing/src/test/java/io/druid/segment/BitmapOffsetTest.java similarity index 87% rename from processing/src/test/java/io/druid/segment/ConciseOffsetTest.java rename to processing/src/test/java/io/druid/segment/BitmapOffsetTest.java index 4e2a2085c82..92f24e403a3 100644 --- a/processing/src/test/java/io/druid/segment/ConciseOffsetTest.java +++ b/processing/src/test/java/io/druid/segment/BitmapOffsetTest.java @@ -19,6 +19,8 @@ package io.druid.segment; +import com.metamx.collections.bitmap.ConciseBitmapFactory; +import com.metamx.collections.bitmap.WrappedImmutableConciseBitmap; import io.druid.segment.data.Offset; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; @@ -27,7 +29,7 @@ import org.junit.Test; /** */ -public class ConciseOffsetTest +public class BitmapOffsetTest { @Test public void testSanity() throws Exception @@ -40,7 +42,7 @@ public class ConciseOffsetTest ImmutableConciseSet set = ImmutableConciseSet.newImmutableFromMutable(mutableSet); - ConciseOffset offset = new ConciseOffset(set); + BitmapOffset offset = new BitmapOffset(new ConciseBitmapFactory(), new WrappedImmutableConciseBitmap(set)); int count = 0; while (offset.withinBounds()) { diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index d653a367155..ae781ca626c 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -21,6 +21,7 @@ package io.druid.segment; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.metamx.collections.bitmap.ConciseBitmapFactory; import io.druid.granularity.QueryGranularity; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; @@ -47,9 +48,18 @@ public class EmptyIndexTest } tmpDir.deleteOnExit(); - IncrementalIndex emptyIndex = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0], TestQueryRunners.pool); - IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(new Interval("2012-08-01/P3D"), emptyIndex); - IndexMaker.merge( + IncrementalIndex emptyIndex = new IncrementalIndex( + 0, + QueryGranularity.NONE, + new AggregatorFactory[0], + TestQueryRunners.pool + ); + IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter( + new Interval("2012-08-01/P3D"), + emptyIndex, + new ConciseBitmapFactory() + ); + IndexMerger.merge( Lists.newArrayList(emptyIndexAdapter), new AggregatorFactory[0], tmpDir diff --git a/processing/src/test/java/io/druid/segment/IndexMakerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java similarity index 89% rename from processing/src/test/java/io/druid/segment/IndexMakerTest.java rename to processing/src/test/java/io/druid/segment/IndexMergerTest.java index 4b39d515a6b..b940bf21ecf 100644 --- a/processing/src/test/java/io/druid/segment/IndexMakerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -39,8 +39,12 @@ import java.util.Arrays; /** */ -public class IndexMakerTest +public class IndexMergerTest { + static { + + } + @Test public void testPersistCaseInsensitive() throws Exception { @@ -50,7 +54,7 @@ public class IndexMakerTest final File tempDir = Files.createTempDir(); try { - QueryableIndex index = IndexIO.loadIndex(IndexMaker.persist(toPersist, tempDir)); + QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir)); Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); @@ -89,20 +93,20 @@ public class IndexMakerTest final File tempDir2 = Files.createTempDir(); final File mergedDir = Files.createTempDir(); try { - QueryableIndex index1 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tempDir1)); + QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1)); Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); Assert.assertEquals(2, index1.getColumnNames().size()); - QueryableIndex index2 = IndexIO.loadIndex(IndexMaker.persist(toPersist2, tempDir2)); + QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2)); Assert.assertEquals(2, index2.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions())); Assert.assertEquals(2, index2.getColumnNames().size()); QueryableIndex merged = IndexIO.loadIndex( - IndexMaker.mergeQueryableIndex( + IndexMerger.mergeQueryableIndex( Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir @@ -146,10 +150,10 @@ public class IndexMakerTest ) ); - final QueryableIndex index1 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tmpDir1)); - final QueryableIndex index2 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tmpDir2)); + final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1)); + final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2)); final QueryableIndex merged = IndexIO.loadIndex( - IndexMaker.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3) + IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3) ); Assert.assertEquals(1, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndex.java b/processing/src/test/java/io/druid/segment/SchemalessIndex.java index 823fd83774a..84283c8082e 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessIndex.java +++ b/processing/src/test/java/io/druid/segment/SchemalessIndex.java @@ -179,11 +179,11 @@ public class SchemalessIndex mergedFile.mkdirs(); mergedFile.deleteOnExit(); - IndexMaker.persist(top, topFile); - IndexMaker.persist(bottom, bottomFile); + IndexMerger.persist(top, topFile); + IndexMerger.persist(bottom, bottomFile); mergedIndex = io.druid.segment.IndexIO.loadIndex( - IndexMaker.mergeQueryableIndex( + IndexMerger.mergeQueryableIndex( Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile ) ); @@ -225,7 +225,7 @@ public class SchemalessIndex mergedFile.deleteOnExit(); QueryableIndex index = IndexIO.loadIndex( - IndexMaker.mergeQueryableIndex( + IndexMerger.mergeQueryableIndex( Arrays.asList(rowPersistedIndexes.get(index1), rowPersistedIndexes.get(index2)), METRIC_AGGS, mergedFile ) ); @@ -262,7 +262,7 @@ public class SchemalessIndex } QueryableIndex index = IndexIO.loadIndex( - IndexMaker.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile) + IndexMerger.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile) ); return index; @@ -343,7 +343,7 @@ public class SchemalessIndex tmpFile.mkdirs(); tmpFile.deleteOnExit(); - IndexMaker.persist(rowIndex, tmpFile); + IndexMerger.persist(rowIndex, tmpFile); rowPersistedIndexes.add(IndexIO.loadIndex(tmpFile)); } } @@ -403,7 +403,7 @@ public class SchemalessIndex theFile.mkdirs(); theFile.deleteOnExit(); filesToMap.add(theFile); - IndexMaker.persist(index, theFile); + IndexMerger.persist(index, theFile); } return filesToMap; @@ -463,7 +463,7 @@ public class SchemalessIndex ); } - return IndexIO.loadIndex(IndexMaker.append(adapters, mergedFile)); + return IndexIO.loadIndex(IndexMerger.append(adapters, mergedFile)); } catch (IOException e) { throw Throwables.propagate(e); @@ -482,7 +482,7 @@ public class SchemalessIndex List filesToMap = makeFilesToMap(tmpFile, files); return IndexIO.loadIndex( - IndexMaker.mergeQueryableIndex( + IndexMerger.mergeQueryableIndex( Lists.newArrayList( Iterables.transform( filesToMap, diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java index 29548d8b55e..23dac58ee4c 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java @@ -1169,6 +1169,7 @@ public class SchemalessTestFull ) ); + /* Uncomment when Druid support for nulls/empty strings is actually consistent List> expectedTopNResults = Arrays.asList( new Result( new DateTime("2011-01-12T00:00:00.000Z"), @@ -1205,6 +1206,43 @@ public class SchemalessTestFull ) ) ); + */ + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("market", "spot") + .put("rows", 4L) + .put("index", 400.0D) + .put("addRowsIndexConstant", 405.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + ImmutableMap.builder() + .put("market", "") + .put("rows", 3L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 204.0D) + .put("uniques", 0.0) + .put("maxIndex", 100.0) + .put("minIndex", 0.0) + .build(), + ImmutableMap.builder() + .put("market", "total_market") + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); List> expectedFilteredTopNResults = Arrays.asList( new Result( diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 466eb1c8ec4..aff48d0cf60 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -132,11 +132,11 @@ public class TestIndex mergedFile.mkdirs(); mergedFile.deleteOnExit(); - IndexMaker.persist(top, DATA_INTERVAL, topFile); - IndexMaker.persist(bottom, DATA_INTERVAL, bottomFile); + IndexMerger.persist(top, DATA_INTERVAL, topFile); + IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile); mergedRealtime = IndexIO.loadIndex( - IndexMaker.mergeQueryableIndex( + IndexMerger.mergeQueryableIndex( Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile @@ -243,7 +243,7 @@ public class TestIndex someTmpFile.mkdirs(); someTmpFile.deleteOnExit(); - IndexMaker.persist(index, someTmpFile); + IndexMerger.persist(index, someTmpFile); return IndexIO.loadIndex(someTmpFile); } catch (IOException e) { diff --git a/processing/src/test/java/io/druid/segment/data/BitmapCreationBenchmark.java b/processing/src/test/java/io/druid/segment/data/BitmapCreationBenchmark.java new file mode 100644 index 00000000000..baa4a9d7f8d --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/BitmapCreationBenchmark.java @@ -0,0 +1,174 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ +package io.druid.segment.data; + +import com.carrotsearch.junitbenchmarks.AbstractBenchmark; +import com.carrotsearch.junitbenchmarks.BenchmarkOptions; +import com.google.common.collect.ImmutableList; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; +import com.metamx.collections.bitmap.MutableBitmap; +import com.metamx.common.logger.Logger; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Random; + +/** + * + */ +@RunWith(Parameterized.class) +public class BitmapCreationBenchmark extends AbstractBenchmark +{ + private static final Logger log = new Logger(BitmapCreationBenchmark.class); + + @Parameterized.Parameters + public static List[]> factoryClasses() + { + return Arrays.[]>asList( + (Class[]) Arrays.>asList( + ConciseBitmapSerdeFactory.class + ).toArray(), + (Class[]) Arrays.>asList( + RoaringBitmapSerdeFactory.class + ).toArray() + ); + } + + final BitmapFactory factory; + final ObjectStrategy objectStrategy; + + public BitmapCreationBenchmark(Class clazz) + throws IllegalAccessException, InstantiationException + { + BitmapSerdeFactory serdeFactory = clazz.newInstance(); + factory = serdeFactory.getBitmapFactory(); + objectStrategy = serdeFactory.getObjectStrategy(); + } + + private static final int numBits = 100000; + + + static Random random; + static int[] randIndex = new int[numBits]; + + @AfterClass + public static void cleanupAfterClass() + { + List[]> classes = factoryClasses(); + for (int i = 0; i < classes.size(); ++i) { + log.info("Entry [%d] is %s", i, classes.get(i)[0].getCanonicalName()); + } + } + + @BeforeClass + public static void setupBeforeClass() + { + for (int i = 0; i < numBits; ++i) { + randIndex[i] = i; + } + // Random seed chosen by hitting keyboard with BOTH hands... multiple times! + random = new Random(78591378); + for (int i = 0; i < numBits; ++i) { + int idex = random.nextInt(randIndex.length); + int swap = randIndex[i]; + randIndex[i] = randIndex[idex]; + randIndex[idex] = swap; + } + } + + ImmutableBitmap baseImmutableBitmap; + MutableBitmap baseMutableBitmap; + byte[] baseBytes; + ByteBuffer baseByteBuffer; + + @Before + public void setup() + { + baseMutableBitmap = factory.makeEmptyMutableBitmap(); + for (int i = 0; i < numBits; ++i) { + baseMutableBitmap.add(i); + } + baseImmutableBitmap = factory.makeImmutableBitmap(baseMutableBitmap); + baseBytes = baseImmutableBitmap.toBytes(); + baseByteBuffer = ByteBuffer.wrap(baseBytes); + } + + + @BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000) + @Test + public void testLinearAddition() + { + MutableBitmap mutableBitmap = factory.makeEmptyMutableBitmap(); + for (int i = 0; i < numBits; ++i) { + mutableBitmap.add(i); + } + Assert.assertEquals(numBits, mutableBitmap.size()); + } + + @BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 10) + @Test + public void testRandomAddition() + { + MutableBitmap mutableBitmap = factory.makeEmptyMutableBitmap(); + for (int i : randIndex) { + mutableBitmap.add(i); + } + Assert.assertEquals(numBits, mutableBitmap.size()); + } + + @BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000) + @Test + public void testLinearAdditionDescending() + { + MutableBitmap mutableBitmap = factory.makeEmptyMutableBitmap(); + for (int i = numBits - 1; i >= 0; --i) { + mutableBitmap.add(i); + } + Assert.assertEquals(numBits, mutableBitmap.size()); + } + + + @BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000) + @Test + public void testToImmutableByteArray() + { + ImmutableBitmap immutableBitmap = factory.makeImmutableBitmap(baseMutableBitmap); + Assert.assertArrayEquals(baseBytes, immutableBitmap.toBytes()); + } + + + @BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000) + @Test + public void testFromImmutableByteArray() + { + ImmutableBitmap immutableBitmap = factory.mapImmutableBitmap(baseByteBuffer); + Assert.assertEquals(numBits, immutableBitmap.size()); + } + +} diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java index 0982f76f8bf..c1a55700b05 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java @@ -65,15 +65,15 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest Closeables.close(indexed, false); } - private void setupSimple() + private void setupSimple(final int chunkSize) { vals = new float[]{ - 0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f + 0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f, 0.16f }; supplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( FloatBuffer.wrap(vals), - 5, + chunkSize, ByteOrder.nativeOrder(), compressionStrategy ); @@ -81,15 +81,15 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest indexed = supplier.get(); } - private void setupSimpleWithSerde() throws IOException + private void setupSimpleWithSerde(final int chunkSize) throws IOException { vals = new float[]{ - 0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f + 0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f, 0.16f }; ByteArrayOutputStream baos = new ByteArrayOutputStream(); final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( - FloatBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), compressionStrategy + FloatBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy ); theSupplier.writeToChannel(Channels.newChannel(baos)); @@ -103,7 +103,7 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest @Test public void testSanity() throws Exception { - setupSimple(); + setupSimple(5); Assert.assertEquals(4, supplier.getBaseFloatBuffers().size()); @@ -111,12 +111,23 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest for (int i = 0; i < indexed.size(); ++i) { Assert.assertEquals(vals[i], indexed.get(i), 0.0); } + + // test powers of 2 + setupSimple(2); + + Assert.assertEquals(9, supplier.getBaseFloatBuffers().size()); + + Assert.assertEquals(vals.length, indexed.size()); + for (int i = 0; i < indexed.size(); ++i) { + Assert.assertEquals(vals[i], indexed.get(i), 0.0); + } + } @Test public void testBulkFill() throws Exception { - setupSimple(); + setupSimple(5); tryFill(0, 15); tryFill(3, 6); @@ -127,14 +138,14 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest @Test(expected = IndexOutOfBoundsException.class) public void testBulkFillTooMuch() throws Exception { - setupSimple(); - tryFill(7, 10); + setupSimple(5); + tryFill(7, 11); } @Test public void testSanityWithSerde() throws Exception { - setupSimpleWithSerde(); + setupSimpleWithSerde(5); Assert.assertEquals(4, supplier.getBaseFloatBuffers().size()); @@ -142,12 +153,22 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest for (int i = 0; i < indexed.size(); ++i) { Assert.assertEquals(vals[i], indexed.get(i), 0.0); } + + // test powers of 2 + setupSimpleWithSerde(2); + + Assert.assertEquals(9, supplier.getBaseFloatBuffers().size()); + + Assert.assertEquals(vals.length, indexed.size()); + for (int i = 0; i < indexed.size(); ++i) { + Assert.assertEquals(vals[i], indexed.get(i), 0.0); + } } @Test public void testBulkFillWithSerde() throws Exception { - setupSimpleWithSerde(); + setupSimpleWithSerde(5); tryFill(0, 15); tryFill(3, 6); @@ -158,8 +179,8 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest @Test(expected = IndexOutOfBoundsException.class) public void testBulkFillTooMuchWithSerde() throws Exception { - setupSimpleWithSerde(); - tryFill(7, 10); + setupSimpleWithSerde(5); + tryFill(7, 11); } // This test attempts to cause a race condition with the DirectByteBuffers, it's non-deterministic in causing it, @@ -167,7 +188,7 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest @Test public void testConcurrentThreadReads() throws Exception { - setupSimple(); + setupSimple(5); final AtomicReference reason = new AtomicReference("none"); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index 4e7ae7d3163..b4c94e18dfb 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -46,7 +46,7 @@ import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.segment.IncrementalIndexSegment; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMaker; +import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; @@ -232,7 +232,7 @@ public class SpatialFilterBonusTest tmpFile.mkdirs(); tmpFile.deleteOnExit(); - IndexMaker.persist(theIndex, tmpFile); + IndexMerger.persist(theIndex, tmpFile); return IndexIO.loadIndex(tmpFile); } @@ -412,12 +412,12 @@ public class SpatialFilterBonusTest mergedFile.mkdirs(); mergedFile.deleteOnExit(); - IndexMaker.persist(first, DATA_INTERVAL, firstFile); - IndexMaker.persist(second, DATA_INTERVAL, secondFile); - IndexMaker.persist(third, DATA_INTERVAL, thirdFile); + IndexMerger.persist(first, DATA_INTERVAL, firstFile); + IndexMerger.persist(second, DATA_INTERVAL, secondFile); + IndexMerger.persist(third, DATA_INTERVAL, thirdFile); QueryableIndex mergedRealtime = IndexIO.loadIndex( - IndexMaker.mergeQueryableIndex( + IndexMerger.mergeQueryableIndex( Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)), METRIC_AGGS, mergedFile diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 43cec97c472..719d5500983 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -1,49 +1,66 @@ - - 4.0.0 - io.druid.extensions - druid-rabbitmq - druid-rabbitmq - druid-rabbitmq + + 4.0.0 + io.druid.extensions + druid-rabbitmq + druid-rabbitmq + druid-rabbitmq - - io.druid - druid - 0.7.0-SNAPSHOT - + + io.druid + druid + 0.7.0-SNAPSHOT + - - - io.druid - druid-api - - - com.rabbitmq - amqp-client - 3.2.1 - - - net.jodah - lyra - 0.3.1 - + + + io.druid + druid-api + + + com.rabbitmq + amqp-client + 3.2.1 + + + net.jodah + lyra + 0.3.1 + - - - junit - junit - test - - - commons-cli - commons-cli - test - - - io.druid - druid-processing - ${project.parent.version} - test - - + + + junit + junit + test + + + commons-cli + commons-cli + test + + + io.druid + druid-processing + ${project.parent.version} + test + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index bdd0d117b2c..000251c7d40 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -18,76 +18,89 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 - io.druid.extensions - druid-s3-extensions - druid-s3-extensions - druid-s3-extensions + + 4.0.0 + io.druid.extensions + druid-s3-extensions + druid-s3-extensions + druid-s3-extensions - - io.druid - druid - 0.7.0-SNAPSHOT - + + io.druid + druid + 0.7.0-SNAPSHOT + - - - io.druid - druid-api - - - - net.java.dev.jets3t - jets3t - - - com.amazonaws - aws-java-sdk - - - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - com.metamx - emitter - - - commons-io - commons-io - + + + io.druid + druid-api + + + + net.java.dev.jets3t + jets3t + + + com.amazonaws + aws-java-sdk + + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + com.metamx + emitter + + + commons-io + commons-io + - - - junit - junit - test - - - org.easymock - easymock - test - - - - - - maven-jar-plugin - - - - true - true - - - - - - + + + junit + junit + test + + + org.easymock + easymock + test + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + true + true + + + + + + diff --git a/server/pom.xml b/server/pom.xml index 8485e146fc0..cf1fae66ad1 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -20,218 +20,223 @@ - 4.0.0 + 4.0.0 + io.druid + druid-server + druid-server + Druid Server + + io.druid - druid-server - druid-server - Druid Server + druid + 0.7.0-SNAPSHOT + - - io.druid - druid - 0.7.0-SNAPSHOT - + + + io.druid + druid-processing + ${project.parent.version} + + + com.metamx + http-client + + + com.metamx + server-metrics + + + commons-cli + commons-cli + + + commons-lang + commons-lang + + + javax.inject + javax.inject + + + org.glassfish + javax.el + + + com.amazonaws + aws-java-sdk + + + org.apache.curator + curator-framework + + + org.apache.curator + curator-x-discovery + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + + + com.sun.jersey + jersey-server + + + com.sun.jersey + jersey-core + + + com.google.inject.extensions + guice-servlet + + + com.sun.jersey.contribs + jersey-guice + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-proxy + + + com.google.code.findbugs + jsr305 + + + io.tesla.aether + tesla-aether + + + org.eclipse.aether + aether-api + + + org.antlr + antlr4-runtime + + + com.google.code.simple-spring-memcached + spymemcached + + + net.jpountz.lz4 + lz4 + + + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-servlets + + + com.ircclouds.irc + irc-api + + + com.maxmind.geoip2 + geoip2 + + + org.apache.derby + derby + 10.11.1.1 + + + org.apache.derby + derbynet + + + org.apache.derby + derbyclient + - - - io.druid - druid-processing - ${project.parent.version} - - - com.metamx - http-client - - - com.metamx - server-metrics - - - commons-cli - commons-cli - - - commons-lang - commons-lang - - - javax.inject - javax.inject - - - org.glassfish - javax.el - - - com.amazonaws - aws-java-sdk - - - org.apache.curator - curator-framework - - - org.apache.curator - curator-x-discovery - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-smile-provider - - - com.fasterxml.jackson.dataformat - jackson-dataformat-smile - - - com.sun.jersey - jersey-server - - - com.sun.jersey - jersey-core - - - com.google.inject.extensions - guice-servlet - - - com.sun.jersey.contribs - jersey-guice - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-proxy - - - com.google.code.findbugs - jsr305 - - - io.tesla.aether - tesla-aether - - - org.eclipse.aether - aether-api - - - org.antlr - antlr4-runtime - - - com.google.code.simple-spring-memcached - spymemcached - - - net.jpountz.lz4 - lz4 - - - org.eclipse.jetty - jetty-servlet - - - org.eclipse.jetty - jetty-servlets - - - com.ircclouds.irc - irc-api - - - com.maxmind.geoip2 - geoip2 - - - org.apache.derby - derby - 10.11.1.1 - - - org.apache.derby - derbynet - 10.11.1.1 - - - org.apache.derby - derbyclient - 10.11.1.1 - - + + + junit + junit + test + + + org.easymock + easymock + test + + + io.druid + druid-processing + ${project.parent.version} + test-jar + test + + + org.apache.curator + curator-test + test + + + com.google.caliper + caliper + test + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + test-jar + + + + + + + true + true + + + + + org.antlr antlr4-maven-plugin - 4.0 - - - - - junit - junit - test - - - org.easymock - easymock - test - - - io.druid - druid-processing - ${project.parent.version} - test-jar - test - - - org.apache.curator - curator-test - test - - - com.google.caliper - caliper - test - - - - - - - maven-jar-plugin - - - - test-jar - - - - - - - true - true - - - - - - org.antlr - antlr4-maven-plugin - - - - antlr4 - - - - - - + + + + antlr4 + + + + + + diff --git a/server/src/main/java/io/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java index c0403c129e7..543951df1c3 100644 --- a/server/src/main/java/io/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -31,6 +31,7 @@ import io.druid.client.selector.ServerSelector; import io.druid.client.selector.TierSelectorStrategy; import io.druid.concurrent.Execs; import io.druid.guice.annotations.Client; +import io.druid.guice.annotations.Smile; import io.druid.query.DataSource; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChestWarehouse; @@ -69,7 +70,7 @@ public class BrokerServerView implements TimelineServerView public BrokerServerView( QueryToolChestWarehouse warehouse, QueryWatcher queryWatcher, - ObjectMapper smileMapper, + @Smile ObjectMapper smileMapper, @Client HttpClient httpClient, ServerInventoryView baseView, TierSelectorStrategy tierSelectorStrategy diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 10af6f0d8e7..8a43cf6ddfb 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -25,6 +25,7 @@ import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -35,6 +36,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; import com.metamx.common.Pair; import com.metamx.common.guava.BaseSequence; +import com.metamx.common.guava.Comparators; import com.metamx.common.guava.LazySequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; @@ -43,6 +45,7 @@ import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; import io.druid.client.selector.QueryableDruidServer; import io.druid.client.selector.ServerSelector; +import io.druid.common.utils.JodaUtils; import io.druid.guice.annotations.Smile; import io.druid.query.BySegmentResultValueClass; import io.druid.query.CacheStrategy; @@ -121,7 +124,7 @@ public class CachingClusteredClient implements QueryRunner final Map> serverSegments = Maps.newTreeMap(); - final List> cachedResults = Lists.newArrayList(); + final List> cachedResults = Lists.newArrayList(); final Map cachePopulatorMap = Maps.newHashMap(); final boolean useCache = query.getContextUseCache(true) @@ -214,7 +217,7 @@ public class CachingClusteredClient implements QueryRunner if (cachedValue != null) { // remove cached segment from set of segments to query segments.remove(segment); - cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue)); + cachedResults.add(Pair.of(segmentQueryInterval, cachedValue)); } else if (populateCache) { final String segmentIdentifier = segment.lhs.getSegment().getIdentifier(); cachePopulatorMap.put( @@ -250,35 +253,14 @@ public class CachingClusteredClient implements QueryRunner @Override public Sequence get() { - ArrayList>> listOfSequences = Lists.newArrayList(); + ArrayList>> sequencesByInterval = Lists.newArrayList(); + addSequencesFromCache(sequencesByInterval); + addSequencesFromServer(sequencesByInterval); - addSequencesFromServer(listOfSequences); - addSequencesFromCache(listOfSequences); - - Collections.sort( - listOfSequences, - Ordering.natural().onResultOf(Pair.>lhsFn()) - ); - - final List> orderedSequences = Lists.newLinkedList(); - DateTime unorderedStart = null; - List> unordered = Lists.newLinkedList(); - for (Pair> sequencePair : listOfSequences) { - if (unorderedStart != null && unorderedStart.getMillis() != sequencePair.lhs.getMillis()) { - orderedSequences.add(toolChest.mergeSequencesUnordered(Sequences.simple(unordered))); - unordered = Lists.newLinkedList(); - } - unorderedStart = sequencePair.lhs; - unordered.add(sequencePair.rhs); - } - if(!unordered.isEmpty()) { - orderedSequences.add(toolChest.mergeSequencesUnordered(Sequences.simple(unordered))); - } - - return toolChest.mergeSequences(Sequences.simple(orderedSequences)); + return mergeCachedAndUncachedSequences(sequencesByInterval, toolChest); } - private void addSequencesFromCache(ArrayList>> listOfSequences) + private void addSequencesFromCache(ArrayList>> listOfSequences) { if (strategy == null) { return; @@ -286,7 +268,7 @@ public class CachingClusteredClient implements QueryRunner final Function pullFromCacheFunction = strategy.pullFromCache(); final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); - for (Pair cachedResultPair : cachedResults) { + for (Pair cachedResultPair : cachedResults) { final byte[] cachedResult = cachedResultPair.rhs; Sequence cachedSequence = new BaseSequence<>( new BaseSequence.IteratorMaker>() @@ -320,7 +302,7 @@ public class CachingClusteredClient implements QueryRunner } @SuppressWarnings("unchecked") - private void addSequencesFromServer(ArrayList>> listOfSequences) + private void addSequencesFromServer(ArrayList>> listOfSequences) { for (Map.Entry> entry : serverSegments.entrySet()) { final DruidServer server = entry.getKey(); @@ -396,13 +378,60 @@ public class CachingClusteredClient implements QueryRunner ); } - listOfSequences.add(Pair.of(intervals.get(0).getStart(), resultSeqToAdd)); + + listOfSequences.add( + Pair.of( + new Interval(intervals.get(0).getStart(), intervals.get(intervals.size() - 1).getEnd()), + resultSeqToAdd + ) + ); } } } ); } + protected Sequence mergeCachedAndUncachedSequences( + List>> sequencesByInterval, + QueryToolChest> toolChest + ) + { + if(sequencesByInterval.isEmpty()) { + return Sequences.empty(); + } + + Collections.sort( + sequencesByInterval, + Ordering.from(Comparators.intervalsByStartThenEnd()).onResultOf(Pair.>lhsFn()) + ); + + // result sequences from overlapping intervals could start anywhere within that interval + // therefore we cannot assume any ordering with respect to the first result from each + // and must resort to calling toolchest.mergeSequencesUnordered for those. + Iterator>> iterator = sequencesByInterval.iterator(); + Pair> current = iterator.next(); + + final List> orderedSequences = Lists.newLinkedList(); + List> unordered = Lists.newLinkedList(); + + unordered.add(current.rhs); + + while(iterator.hasNext()) { + Pair> next = iterator.next(); + if(!next.lhs.overlaps(current.lhs)) { + orderedSequences.add(toolChest.mergeSequencesUnordered(Sequences.simple(unordered))); + unordered = Lists.newLinkedList(); + } + unordered.add(next.rhs); + current = next; + } + if(!unordered.isEmpty()) { + orderedSequences.add(toolChest.mergeSequencesUnordered(Sequences.simple(unordered))); + } + + return toolChest.mergeSequencesUnordered(Sequences.simple(orderedSequences)); + } + private static class CachePopulator { private final Cache cache; diff --git a/server/src/main/java/io/druid/client/FilteredServerViewProvider.java b/server/src/main/java/io/druid/client/FilteredServerViewProvider.java index f9b20106ba2..4570c7e48cb 100644 --- a/server/src/main/java/io/druid/client/FilteredServerViewProvider.java +++ b/server/src/main/java/io/druid/client/FilteredServerViewProvider.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.inject.Provider; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = SingleServerInventoryProvider.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = FilteredBatchServerViewProvider.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "legacy", value = FilteredSingleServerViewProvider.class), @JsonSubTypes.Type(name = "batch", value = FilteredBatchServerViewProvider.class) diff --git a/server/src/main/java/io/druid/client/ServerInventoryView.java b/server/src/main/java/io/druid/client/ServerInventoryView.java index 2481c29b55c..b95dda08c91 100644 --- a/server/src/main/java/io/druid/client/ServerInventoryView.java +++ b/server/src/main/java/io/druid/client/ServerInventoryView.java @@ -22,6 +22,7 @@ package io.druid.client; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.MapMaker; @@ -36,6 +37,8 @@ import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.CharBuffer; import java.util.Map; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; @@ -111,6 +114,8 @@ public abstract class ServerInventoryView implements ServerView, return jsonMapper.readValue(bytes, typeReference); } catch (IOException e) { + CharBuffer charBuffer = Charsets.UTF_8.decode(ByteBuffer.wrap(bytes)); + log.error(e, "Could not parse json: %s", charBuffer.toString()); throw Throwables.propagate(e); } } diff --git a/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java index fa48fba4661..f506d03148a 100644 --- a/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java @@ -25,7 +25,7 @@ import com.google.inject.Provider; /** */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = SingleServerInventoryProvider.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = BatchServerInventoryViewProvider.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "legacy", value = SingleServerInventoryProvider.class), @JsonSubTypes.Type(name = "batch", value = BatchServerInventoryViewProvider.class) diff --git a/server/src/main/java/io/druid/curator/CuratorConfig.java b/server/src/main/java/io/druid/curator/CuratorConfig.java index 50d23f64104..3e8795bfd2c 100644 --- a/server/src/main/java/io/druid/curator/CuratorConfig.java +++ b/server/src/main/java/io/druid/curator/CuratorConfig.java @@ -27,6 +27,7 @@ import org.skife.config.Default; public abstract class CuratorConfig { @Config("druid.zk.service.host") + @Default("localhost") public abstract String getZkHosts(); @Config("druid.zk.service.sessionTimeoutMs") diff --git a/server/src/main/java/io/druid/guice/MetadataDbConfigModule.java b/server/src/main/java/io/druid/guice/MetadataConfigModule.java similarity index 96% rename from server/src/main/java/io/druid/guice/MetadataDbConfigModule.java rename to server/src/main/java/io/druid/guice/MetadataConfigModule.java index 71b27b9793f..be08f43fa72 100644 --- a/server/src/main/java/io/druid/guice/MetadataDbConfigModule.java +++ b/server/src/main/java/io/druid/guice/MetadataConfigModule.java @@ -21,12 +21,12 @@ package io.druid.guice; import com.google.inject.Binder; import com.google.inject.Module; -import io.druid.metadata.MetadataStorageConnectorConfig; import io.druid.metadata.MetadataRuleManagerConfig; import io.druid.metadata.MetadataSegmentManagerConfig; +import io.druid.metadata.MetadataStorageConnectorConfig; import io.druid.metadata.MetadataStorageTablesConfig; -public class MetadataDbConfigModule implements Module +public class MetadataConfigModule implements Module { @Override public void configure(Binder binder) diff --git a/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java b/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java index f69c68e0119..fef1fb04d95 100644 --- a/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java +++ b/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java @@ -34,6 +34,8 @@ import io.druid.metadata.MetadataSegmentManagerProvider; import io.druid.metadata.MetadataSegmentPublisher; import io.druid.metadata.MetadataSegmentPublisherProvider; import io.druid.metadata.MetadataStorageConnector; +import io.druid.metadata.MetadataStorageProvider; +import io.druid.metadata.NoopMetadataStorageProvider; import io.druid.metadata.SQLMetadataConnector; import io.druid.metadata.SQLMetadataRuleManager; import io.druid.metadata.SQLMetadataRuleManagerProvider; @@ -62,6 +64,13 @@ public class SQLMetadataStorageDruidModule implements Module PolyBind.createChoiceWithDefault( binder, PROPERTY, Key.get(MetadataStorageConnector.class), null, defaultPropertyValue ); + PolyBind.createChoiceWithDefault( + binder, + PROPERTY, + Key.get(MetadataStorageProvider.class), + Key.get(NoopMetadataStorageProvider.class), + defaultPropertyValue + ); PolyBind.createChoiceWithDefault( binder, PROPERTY, Key.get(SQLMetadataConnector.class), null, defaultPropertyValue ); diff --git a/server/src/main/java/io/druid/guice/http/DruidHttpClientConfig.java b/server/src/main/java/io/druid/guice/http/DruidHttpClientConfig.java index cf85087c98c..b8ad2e272b8 100644 --- a/server/src/main/java/io/druid/guice/http/DruidHttpClientConfig.java +++ b/server/src/main/java/io/druid/guice/http/DruidHttpClientConfig.java @@ -32,7 +32,7 @@ public class DruidHttpClientConfig { @JsonProperty @Min(0) - private int numConnections = 5; + private int numConnections = 20; @JsonProperty private Period readTimeout = new Period("PT15M"); diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 32c6e89e93e..058a007701d 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -36,7 +36,7 @@ import io.druid.curator.CuratorModule; import io.druid.curator.discovery.DiscoveryModule; import io.druid.guice.AWSModule; import io.druid.guice.AnnouncerModule; -import io.druid.guice.DerbyMetadataStorageDruidModule; +import io.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule; import io.druid.guice.DruidProcessingModule; import io.druid.guice.DruidSecondaryModule; import io.druid.guice.ExtensionsConfig; @@ -45,7 +45,7 @@ import io.druid.guice.IndexingServiceDiscoveryModule; import io.druid.guice.JacksonConfigManagerModule; import io.druid.guice.LifecycleModule; import io.druid.guice.LocalDataStorageDruidModule; -import io.druid.guice.MetadataDbConfigModule; +import io.druid.guice.MetadataConfigModule; import io.druid.guice.ParsersModule; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; @@ -140,7 +140,7 @@ public class Initialization for (String coordinate : config.getCoordinates()) { log.info("Loading extension[%s] for class[%s]", coordinate, clazz.getName()); try { - URLClassLoader loader = getClassLoaderForCoordinates(aether, coordinate); + URLClassLoader loader = getClassLoaderForCoordinates(aether, coordinate, config.getDefaultVersion()); final ServiceLoader serviceLoader = ServiceLoader.load(clazz, loader); @@ -160,13 +160,28 @@ public class Initialization return retVal; } - public static URLClassLoader getClassLoaderForCoordinates(TeslaAether aether, String coordinate) + public static URLClassLoader getClassLoaderForCoordinates(TeslaAether aether, String coordinate, String defaultVersion) throws DependencyResolutionException, MalformedURLException { URLClassLoader loader = loadersMap.get(coordinate); if (loader == null) { final CollectRequest collectRequest = new CollectRequest(); - collectRequest.setRoot(new Dependency(new DefaultArtifact(coordinate), JavaScopes.RUNTIME)); + + DefaultArtifact versionedArtifact; + try { + // this will throw an exception if no version is specified + versionedArtifact = new DefaultArtifact(coordinate); + } + catch (IllegalArgumentException e) { + // try appending the default version so we can specify artifacts without versions + if (defaultVersion != null) { + versionedArtifact = new DefaultArtifact(coordinate + ":" + defaultVersion); + } else { + throw e; + } + } + + collectRequest.setRoot(new Dependency(versionedArtifact, JavaScopes.RUNTIME)); DependencyRequest dependencyRequest = new DependencyRequest( collectRequest, DependencyFilterUtils.andFilter( @@ -333,7 +348,7 @@ public class Initialization new QueryRunnerFactoryModule(), new DiscoveryModule(), new ServerViewModule(), - new MetadataDbConfigModule(), + new MetadataConfigModule(), new DerbyMetadataStorageDruidModule(), new JacksonConfigManagerModule(), new IndexingServiceDiscoveryModule(), diff --git a/server/src/main/java/io/druid/metadata/MetadataStorage.java b/server/src/main/java/io/druid/metadata/MetadataStorage.java new file mode 100644 index 00000000000..6a53ac251bf --- /dev/null +++ b/server/src/main/java/io/druid/metadata/MetadataStorage.java @@ -0,0 +1,20 @@ +package io.druid.metadata; + +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import io.druid.guice.ManageLifecycle; + +public abstract class MetadataStorage +{ + @LifecycleStart + public void start() + { + // do nothing + } + + @LifecycleStop + public void stop() + { + // do nothing + } +} diff --git a/server/src/main/java/io/druid/metadata/MetadataStorageProvider.java b/server/src/main/java/io/druid/metadata/MetadataStorageProvider.java new file mode 100644 index 00000000000..8c441954ffb --- /dev/null +++ b/server/src/main/java/io/druid/metadata/MetadataStorageProvider.java @@ -0,0 +1,26 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.metadata; + +import com.google.inject.Provider; + +public interface MetadataStorageProvider extends Provider +{ +} diff --git a/server/src/main/java/io/druid/metadata/NoopMetadataStorageProvider.java b/server/src/main/java/io/druid/metadata/NoopMetadataStorageProvider.java new file mode 100644 index 00000000000..278451a36c9 --- /dev/null +++ b/server/src/main/java/io/druid/metadata/NoopMetadataStorageProvider.java @@ -0,0 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.metadata; + +public class NoopMetadataStorageProvider implements MetadataStorageProvider +{ + @Override + public MetadataStorage get() + { + return new MetadataStorage() {}; + } +} diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java index 561ed963b3e..af1274304d8 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java @@ -75,7 +75,9 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector */ protected abstract String getSerialType(); - protected abstract boolean tableExists(Handle handle, final String tableName); + public String getValidationQuery() { return "SELECT 1"; } + + public abstract boolean tableExists(Handle handle, final String tableName); protected boolean isTransientException(Throwable e) { return false; @@ -367,7 +369,7 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector String uri = connectorConfig.getConnectURI(); dataSource.setUrl(uri); - dataSource.setValidationQuery(connectorConfig.getValidationQuery()); + dataSource.setValidationQuery(getValidationQuery()); dataSource.setTestOnBorrow(true); return dataSource; diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManagerProvider.java b/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManagerProvider.java index ab7ca53dda4..47f0c27bc1f 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManagerProvider.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManagerProvider.java @@ -33,7 +33,6 @@ public class SQLMetadataSegmentManagerProvider implements MetadataSegmentManager private final Supplier config; private final Supplier storageConfig; private final SQLMetadataConnector connector; - private final IDBI dbi; private final Lifecycle lifecycle; @Inject @@ -49,7 +48,6 @@ public class SQLMetadataSegmentManagerProvider implements MetadataSegmentManager this.config = config; this.storageConfig = storageConfig; this.connector = connector; - this.dbi = this.connector.getDBI(); this.lifecycle = lifecycle; } diff --git a/server/src/main/java/io/druid/metadata/DerbyConnector.java b/server/src/main/java/io/druid/metadata/storage/derby/DerbyConnector.java similarity index 67% rename from server/src/main/java/io/druid/metadata/DerbyConnector.java rename to server/src/main/java/io/druid/metadata/storage/derby/DerbyConnector.java index 70ba42f7fc6..e15c21472c9 100644 --- a/server/src/main/java/io/druid/metadata/DerbyConnector.java +++ b/server/src/main/java/io/druid/metadata/storage/derby/DerbyConnector.java @@ -17,17 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.metadata; +package io.druid.metadata.storage.derby; -import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.google.common.base.Supplier; import com.google.inject.Inject; -import org.apache.derby.drda.NetworkServerControl; +import io.druid.metadata.MetadataStorageConnectorConfig; +import io.druid.metadata.MetadataStorageTablesConfig; +import io.druid.metadata.SQLMetadataConnector; +import org.apache.commons.dbcp2.BasicDataSource; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.tweak.ConnectionFactory; - -import java.net.InetAddress; public class DerbyConnector extends SQLMetadataConnector { @@ -37,17 +36,27 @@ public class DerbyConnector extends SQLMetadataConnector @Inject public DerbyConnector(Supplier config, Supplier dbTables) { - this(config, dbTables, new DBI(getConnectionFactory("druidDerbyDb"))); + super(config, dbTables); + + final BasicDataSource datasource = getDatasource(); + datasource.setDriverClassLoader(getClass().getClassLoader()); + datasource.setDriverClassName("org.apache.derby.jdbc.ClientDriver"); + + this.dbi = new DBI(datasource); } - public DerbyConnector(Supplier config, Supplier dbTables, DBI dbi) + public DerbyConnector( + Supplier config, + Supplier dbTables, + DBI dbi + ) { super(config, dbTables); this.dbi = dbi; } @Override - protected boolean tableExists(Handle handle, String tableName) + public boolean tableExists(Handle handle, String tableName) { return !handle.createQuery("select * from SYS.SYSTABLES where tablename = :tableName") .bind("tableName", tableName.toUpperCase()) @@ -64,14 +73,6 @@ public class DerbyConnector extends SQLMetadataConnector @Override public DBI getDBI() { return dbi; } - private static ConnectionFactory getConnectionFactory(String dbName) - { - try { - NetworkServerControl server = new NetworkServerControl(InetAddress.getByName("localhost"),1527); - server.start(null); - } catch (Exception e) { - throw Throwables.propagate(e); - } - return new DerbyConnectionFactory(dbName); - } + @Override + public String getValidationQuery() { return "VALUES 1"; } } diff --git a/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorage.java b/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorage.java new file mode 100644 index 00000000000..eea0757f3a3 --- /dev/null +++ b/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorage.java @@ -0,0 +1,78 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.metadata.storage.derby; + +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import io.druid.guice.ManageLifecycle; +import io.druid.metadata.MetadataStorage; +import io.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.derby.drda.NetworkServerControl; + +import java.net.InetAddress; + + +@ManageLifecycle +public class DerbyMetadataStorage extends MetadataStorage +{ + private static final Logger log = new Logger(DerbyMetadataStorage.class); + + private final NetworkServerControl server; + + @Inject + public DerbyMetadataStorage(MetadataStorageConnectorConfig config) + { + try { + this.server = new NetworkServerControl(InetAddress.getByName(config.getHost()), config.getPort()); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + } + + @Override + @LifecycleStart + public void start() + { + try { + log.info("Starting Derby Metadata Storage"); + server.start(null); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @Override + @LifecycleStop + public void stop() + { + try { + server.shutdown(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/server/src/main/java/io/druid/guice/DerbyMetadataStorageDruidModule.java b/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageDruidModule.java similarity index 80% rename from server/src/main/java/io/druid/guice/DerbyMetadataStorageDruidModule.java rename to server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageDruidModule.java index 5dfed2ca4a6..d95e822d7b3 100644 --- a/server/src/main/java/io/druid/guice/DerbyMetadataStorageDruidModule.java +++ b/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageDruidModule.java @@ -17,15 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.guice; +package io.druid.metadata.storage.derby; import com.google.inject.Binder; import com.google.inject.Key; -import com.google.inject.Provides; -import io.druid.metadata.DerbyConnector; +import io.druid.guice.LazySingleton; +import io.druid.guice.PolyBind; +import io.druid.guice.SQLMetadataStorageDruidModule; import io.druid.metadata.MetadataStorageConnector; +import io.druid.metadata.MetadataStorageProvider; import io.druid.metadata.SQLMetadataConnector; -import org.skife.jdbi.v2.IDBI; public class DerbyMetadataStorageDruidModule extends SQLMetadataStorageDruidModule { @@ -42,6 +43,11 @@ public class DerbyMetadataStorageDruidModule extends SQLMetadataStorageDruidModu createBindingChoices(binder, TYPE); super.configure(binder); + PolyBind.optionBinder(binder, Key.get(MetadataStorageProvider.class)) + .addBinding(TYPE) + .to(DerbyMetadataStorageProvider.class) + .in(LazySingleton.class); + PolyBind.optionBinder(binder, Key.get(MetadataStorageConnector.class)) .addBinding(TYPE) .to(DerbyConnector.class) diff --git a/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageProvider.java b/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageProvider.java new file mode 100644 index 00000000000..11daddb6065 --- /dev/null +++ b/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageProvider.java @@ -0,0 +1,42 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.metadata.storage.derby; + +import com.google.inject.Inject; +import io.druid.metadata.MetadataStorage; +import io.druid.metadata.MetadataStorageConnectorConfig; +import io.druid.metadata.MetadataStorageProvider; + +public class DerbyMetadataStorageProvider implements MetadataStorageProvider +{ + private final DerbyMetadataStorage storage; + + @Inject + public DerbyMetadataStorageProvider(MetadataStorageConnectorConfig config) + { + this.storage = new DerbyMetadataStorage(config); + } + + @Override + public MetadataStorage get() + { + return storage; + } +} diff --git a/server/src/main/java/io/druid/segment/indexing/DataSchema.java b/server/src/main/java/io/druid/segment/indexing/DataSchema.java index 3013c9728f7..b9f54e49c52 100644 --- a/server/src/main/java/io/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/io/druid/segment/indexing/DataSchema.java @@ -22,7 +22,9 @@ package io.druid.segment.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; +import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.TimestampSpec; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -50,19 +52,28 @@ public class DataSchema final Set dimensionExclusions = Sets.newHashSet(); for (AggregatorFactory aggregator : aggregators) { - dimensionExclusions.add(aggregator.getName()); + dimensionExclusions.addAll(aggregator.requiredFields()); } if (parser != null && parser.getParseSpec() != null) { - if (parser.getParseSpec().getTimestampSpec() != null) { - dimensionExclusions.add(parser.getParseSpec().getTimestampSpec().getTimestampColumn()); + final DimensionsSpec dimensionsSpec = parser.getParseSpec().getDimensionsSpec(); + final TimestampSpec timestampSpec = parser.getParseSpec().getTimestampSpec(); + + // exclude timestamp from dimensions by default, unless explicitly included in the list of dimensions + if (timestampSpec != null) { + final String timestampColumn = timestampSpec.getTimestampColumn(); + if (!(dimensionsSpec.hasCustomDimensions() && dimensionsSpec.getDimensions().contains(timestampColumn))) { + dimensionExclusions.add(timestampColumn); + } } - if (parser.getParseSpec().getDimensionsSpec() != null) { + if (dimensionsSpec != null) { this.parser = parser.withParseSpec( parser.getParseSpec() .withDimensionsSpec( - parser.getParseSpec() - .getDimensionsSpec() - .withDimensionExclusions(dimensionExclusions) + dimensionsSpec + .withDimensionExclusions( + Sets.difference(dimensionExclusions, + Sets.newHashSet(dimensionsSpec.getDimensions())) + ) ) ); } else { diff --git a/server/src/main/java/io/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java b/server/src/main/java/io/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java index 807764e515c..677f7a2d181 100644 --- a/server/src/main/java/io/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java +++ b/server/src/main/java/io/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java @@ -50,6 +50,10 @@ public class ArbitraryGranularitySpec implements GranularitySpec this.queryGranularity = queryGranularity; this.intervals = Sets.newTreeSet(Comparators.intervalsByStartThenEnd()); + if (inputIntervals == null) { + inputIntervals = Lists.newArrayList(); + } + // Insert all intervals for (final Interval inputInterval : inputIntervals) { intervals.add(inputInterval); diff --git a/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java index 717b0b513a0..81dd8d3136f 100644 --- a/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java @@ -22,7 +22,6 @@ package io.druid.segment.loading; import com.metamx.common.logger.Logger; import io.druid.segment.IndexIO; import io.druid.segment.QueryableIndex; -import org.apache.commons.io.FileUtils; import java.io.File; import java.io.IOException; @@ -40,13 +39,6 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory return IndexIO.loadIndex(parentDir); } catch (IOException e) { - log.warn(e, "Got exception!!!! Going to delete parentDir[%s]", parentDir); - try { - FileUtils.deleteDirectory(parentDir); - } - catch (IOException e2) { - log.error(e, "Problem deleting parentDir[%s]", parentDir); - } throw new SegmentLoadingException(e, "%s", e.getMessage()); } } diff --git a/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java b/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java index bfec5093392..b1c3162c100 100644 --- a/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java +++ b/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java @@ -48,6 +48,8 @@ public class OmniSegmentLoader implements SegmentLoader private final List locations; + private final Object lock = new Object(); + @Inject public OmniSegmentLoader( Map pullers, @@ -118,16 +120,33 @@ public class OmniSegmentLoader implements SegmentLoader } File storageDir = new File(loc.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); - if (!storageDir.mkdirs()) { - log.debug("Unable to make parent file[%s]", storageDir); + + // We use a marker to prevent the case where a segment is downloaded, but before the download completes, + // the parent directories of the segment are removed + final File downloadStartMarker = new File(storageDir, "downloadStartMarker"); + synchronized (lock) { + if (!storageDir.mkdirs()) { + log.debug("Unable to make parent file[%s]", storageDir); + } + try { + downloadStartMarker.createNewFile(); + } + catch (IOException e) { + throw new SegmentLoadingException("Unable to create marker file for [%s]", storageDir); + } } getPuller(segment.getLoadSpec()).getSegmentFiles(segment, storageDir); + + if (!downloadStartMarker.delete()) { + throw new SegmentLoadingException("Unable to remove marker file for [%s]", storageDir); + } + + loc.addSegment(segment); retVal = storageDir; - } - else { + } else { retVal = new File(loc.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); } @@ -151,9 +170,10 @@ public class OmniSegmentLoader implements SegmentLoader } try { + // Druid creates folders of the form dataSource/interval/version/partitionNum. + // We need to clean up all these directories if they are all empty. File cacheFile = new File(loc.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); - log.info("Deleting directory[%s]", cacheFile); - FileUtils.deleteDirectory(cacheFile); + cleanupCacheFiles(loc.getPath(), cacheFile); loc.removeSegment(segment); } catch (IOException e) { @@ -172,4 +192,25 @@ public class OmniSegmentLoader implements SegmentLoader return loader; } + + public void cleanupCacheFiles(File baseFile, File cacheFile) throws IOException + { + if (cacheFile.equals(baseFile)) { + return; + } + + synchronized (lock) { + log.info("Deleting directory[%s]", cacheFile); + try { + FileUtils.deleteDirectory(cacheFile); + } + catch (Exception e) { + log.error("Unable to remove file[%s]", cacheFile); + } + } + + if (cacheFile.getParentFile() != null && cacheFile.getParentFile().listFiles().length == 0) { + cleanupCacheFiles(baseFile, cacheFile.getParentFile()); + } + } } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 061e51b1340..b70286fd036 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -325,7 +325,13 @@ public class RealtimePlumber implements Plumber { final Interval interval = sink.getInterval(); - // use a file to indicate that pushing has completed + // Bail out if this sink has been abandoned by a previously-executed task. + if (sinks.get(truncatedTime) != sink) { + log.info("Sink[%s] was abandoned, bailing out of persist-n-merge.", sink); + return; + } + + // Use a file to indicate that pushing has completed. final File persistDir = computePersistDir(schema, interval); final File mergedTarget = new File(persistDir, "merged"); final File isPushedMarker = new File(persistDir, "isPushedMarker"); diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 38ec31c1bc8..8eb670922ca 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -87,7 +87,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker final FinalizeResultsQueryRunner baseRunner = new FinalizeResultsQueryRunner( toolChest.postMergeQueryDecoration( toolChest.mergeResults( - new UnionQueryRunner<>( + new UnionQueryRunner( new MetricsEmittingQueryRunner( emitter, new Function, ServiceMetricEvent.Builder>() @@ -101,6 +101,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker toolChest.preMergeQueryDecoration( new RetryQueryRunner( baseClient, + toolChest, retryConfig, objectMapper ) diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index b66b080f6b9..9a6eca65ef2 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -153,8 +153,8 @@ public class QueryResource log.debug("Got query [%s]", query); } - final Map context = new MapMaker().makeMap(); - final Sequence res = query.run(texasRanger, context); + final Map responseContext = new MapMaker().makeMap(); + final Sequence res = query.run(texasRanger, responseContext); final Sequence results; if (res == null) { results = Sequences.empty(); @@ -209,10 +209,10 @@ public class QueryResource outputStream.close(); } }, - isSmile ? APPLICATION_JSON : APPLICATION_SMILE - ) + isSmile ? APPLICATION_SMILE : APPLICATION_JSON + ) .header("X-Druid-Query-Id", queryId) - .header("X-Druid-Response-Context", jsonMapper.writeValueAsString(context)) + .header("X-Druid-Response-Context", jsonMapper.writeValueAsString(responseContext)) .build(); } catch (Exception e) { diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index e79e4a3a978..f3d22df8fe6 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -421,7 +421,7 @@ public class ServerManager implements QuerySegmentWalker }, new ReferenceCountingSegmentQueryRunner(factory, adapter), "scan/time" - ).withWaitMeasuredFromNow(), + ), cacheConfig ) ) diff --git a/server/src/main/java/io/druid/server/initialization/ServerConfig.java b/server/src/main/java/io/druid/server/initialization/ServerConfig.java index 4491f2e3874..a906ed92abf 100644 --- a/server/src/main/java/io/druid/server/initialization/ServerConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ServerConfig.java @@ -31,8 +31,7 @@ public class ServerConfig { @JsonProperty @Min(1) - // Jetty defaults are whack - private int numThreads = Math.max(10, (Runtime.getRuntime().availableProcessors() * 17) / 16 + 2); + private int numThreads = Math.max(10, (Runtime.getRuntime().availableProcessors() * 17) / 16 + 2) + 30; @JsonProperty @NotNull diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerConfig.java b/server/src/main/java/io/druid/server/router/TieredBrokerConfig.java index 11e8ac67181..ccb8202d98e 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerConfig.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerConfig.java @@ -34,9 +34,12 @@ import java.util.List; */ public class TieredBrokerConfig { + public static final String DEFAULT_COORDINATOR_SERVICE_NAME = "druid/coordinator"; + public static final String DEFAULT_BROKER_SERVICE_NAME = "druid/broker"; + @JsonProperty @NotNull - private String defaultBrokerServiceName = ""; + private String defaultBrokerServiceName = DEFAULT_BROKER_SERVICE_NAME; @JsonProperty private LinkedHashMap tierToBrokerMap; @@ -51,7 +54,7 @@ public class TieredBrokerConfig @JsonProperty @NotNull - private String coordinatorServiceName = null; + private String coordinatorServiceName = DEFAULT_COORDINATOR_SERVICE_NAME; @JsonProperty @NotNull diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 17abc5c3214..57a0cf2ece3 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -35,6 +35,7 @@ import com.google.common.collect.Ordering; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import com.metamx.common.ISE; +import com.metamx.common.Pair; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.MergeIterable; import com.metamx.common.guava.Sequence; @@ -598,6 +599,56 @@ public class CachingClusteredClientTest ); } + @Test + public void testOutOfOrderSequenceMerging() throws Exception + { + List>>> sequences = + Lists.newArrayList( + Pair.of( + // this could ne the result of a historical node returning the merged result of + // a) an empty result for segment 2011-01-02/2011-01-05 + // and b) result for a second partition for 2011-01-05/2011-01-10 + new Interval("2011-01-02/2011-01-10"), + Sequences.simple( + makeTopNResults( + new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ) + ) + ), + + Pair.of( + new Interval("2011-01-05/2011-01-10"), + Sequences.simple( + makeTopNResults( + new DateTime("2011-01-06T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ) + ) + ) + ); + + TestHelper.assertExpectedResults( + makeTopNResults( + new DateTime("2011-01-06T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983, + new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ), + client.mergeCachedAndUncachedSequences( + sequences, + new TopNQueryQueryToolChest(new TopNQueryConfig()) + ) + ); + } + + @Test @SuppressWarnings("unchecked") public void testTopNCachingEmptyResults() throws Exception diff --git a/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 7440cff887c..287f2d7d49a 100644 --- a/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -46,13 +46,7 @@ public class SQLMetadataStorageActionHandlerTest @Before public void setUp() throws Exception { - MetadataStorageConnectorConfig config = jsonMapper.readValue( - "{" - + "\"type\" : \"db\",\n" - + "\"segmentTable\" : \"segments\"\n" - + "}", - MetadataStorageConnectorConfig.class - ); + MetadataStorageConnectorConfig config = new MetadataStorageConnectorConfig(); connector = new TestDerbyConnector( Suppliers.ofInstance(config), diff --git a/server/src/test/java/io/druid/metadata/TestDerbyConnector.java b/server/src/test/java/io/druid/metadata/TestDerbyConnector.java index b45eb0c5070..8b301f331cb 100644 --- a/server/src/test/java/io/druid/metadata/TestDerbyConnector.java +++ b/server/src/test/java/io/druid/metadata/TestDerbyConnector.java @@ -20,6 +20,7 @@ package io.druid.metadata; import com.google.common.base.Supplier; +import io.druid.metadata.storage.derby.DerbyConnector; import org.junit.Assert; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException; diff --git a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java new file mode 100644 index 00000000000..99ecdc43780 --- /dev/null +++ b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java @@ -0,0 +1,87 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.indexing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import org.junit.Assert; +import org.joda.time.Interval; +import org.junit.Test; + +public class DataSchemaTest +{ + @Test + public void testDefaultExclusions() throws Exception + { + DataSchema schema = new DataSchema( + "test", + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("time", "auto"), + new DimensionsSpec(ImmutableList.of("dimB", "dimA"), null, null) + ), + null, null, null, null + ), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2"), + }, + new ArbitraryGranularitySpec(QueryGranularity.DAY, ImmutableList.of(Interval.parse("2014/2015"))) + ); + + Assert.assertEquals( + ImmutableSet.of("time", "col1", "col2"), + schema.getParser().getParseSpec().getDimensionsSpec().getDimensionExclusions() + ); + } + + @Test + public void testExplicitInclude() throws Exception + { + DataSchema schema = new DataSchema( + "test", + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("time", "auto"), + new DimensionsSpec(ImmutableList.of("time", "dimA", "dimB", "col2"), ImmutableList.of("dimC"), null) + ), + null, null, null, null + ), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2"), + }, + new ArbitraryGranularitySpec(QueryGranularity.DAY, ImmutableList.of(Interval.parse("2014/2015"))) + ); + + Assert.assertEquals( + ImmutableSet.of("dimC", "col1"), + schema.getParser().getParseSpec().getDimensionsSpec().getDimensionExclusions() + ); + } +} diff --git a/server/src/test/java/io/druid/server/coordination/CostBalancerStrategyTest.java b/server/src/test/java/io/druid/server/coordination/CostBalancerStrategyTest.java index f46f7f0e1ce..cd12eaf7317 100644 --- a/server/src/test/java/io/druid/server/coordination/CostBalancerStrategyTest.java +++ b/server/src/test/java/io/druid/server/coordination/CostBalancerStrategyTest.java @@ -27,11 +27,12 @@ import io.druid.server.coordinator.CostBalancerStrategy; import io.druid.server.coordinator.LoadQueuePeonTester; import io.druid.server.coordinator.ServerHolder; import io.druid.timeline.DataSegment; -import junit.framework.Assert; +import org.junit.Assert; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; +import org.junit.Ignore; import org.junit.Test; import java.util.List; @@ -109,19 +110,33 @@ public class CostBalancerStrategyTest } @Test - public void testCostBalancerMultithreadStrategy() throws InterruptedException + public void testCostBalancerMultiThreadedStrategy() throws InterruptedException { setupDummyCluster(10, 20); DataSegment segment = getSegment(1000); - BalancerStrategy strategy = new CostBalancerStrategy(DateTime.now(DateTimeZone.UTC), 1); + final DateTime referenceTimestamp = new DateTime("2014-01-01"); + BalancerStrategy strategy = new CostBalancerStrategy(referenceTimestamp, 4); ServerHolder holder = strategy.findNewSegmentHomeReplicator(segment, serverHolderList); Assert.assertNotNull("Should be able to find a place for new segment!!", holder); Assert.assertEquals("Best Server should be BEST_SERVER", "BEST_SERVER", holder.getServer().getName()); } @Test - public void testPerf() throws InterruptedException + public void testCostBalancerSingleThreadStrategy() throws InterruptedException + { + setupDummyCluster(10, 20); + DataSegment segment = getSegment(1000); + + final DateTime referenceTimestamp = new DateTime("2014-01-01"); + BalancerStrategy strategy = new CostBalancerStrategy(referenceTimestamp, 1); + ServerHolder holder = strategy.findNewSegmentHomeReplicator(segment, serverHolderList); + Assert.assertNotNull("Should be able to find a place for new segment!!", holder); + Assert.assertEquals("Best Server should be BEST_SERVER", "BEST_SERVER", holder.getServer().getName()); + } + + @Test @Ignore + public void testBenchmark() throws InterruptedException { setupDummyCluster(100, 500); DataSegment segment = getSegment(1000); @@ -140,10 +155,5 @@ public class CostBalancerStrategyTest System.err.println("Latency - Single Threaded (ms): " + latencySingleThread); System.err.println("Latency - Multi Threaded (ms): " + latencyMultiThread); - - Assert.assertTrue("Latency of multi-thread strategy should always be less than single thread.", latencyMultiThread < latencySingleThread); } - - - } diff --git a/server/src/test/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java index 2b2162148ed..d020a615c46 100644 --- a/server/src/test/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java +++ b/server/src/test/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java @@ -28,6 +28,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.topn.TopNQueryBuilder; import org.junit.Assert; import org.junit.Test; @@ -36,7 +37,7 @@ import java.util.LinkedHashMap; public class JavaScriptTieredBrokerSelectorStrategyTest { final TieredBrokerSelectorStrategy jsStrategy = new JavaScriptTieredBrokerSelectorStrategy( - "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" + "function (config, query) { if (query.getAggregatorSpecs && query.getDimensionSpec && query.getDimensionSpec().getDimension() == 'bigdim' && query.getAggregatorSpecs().size() >= 3) { var size = config.getTierToBrokerMap().values().size(); if (size > 0) { return config.getTierToBrokerMap().values().toArray()[size-1] } else { return config.getDefaultBrokerServiceName() } } else { return null } }" ); @Test @@ -57,7 +58,8 @@ public class JavaScriptTieredBrokerSelectorStrategyTest { final LinkedHashMap tierBrokerMap = new LinkedHashMap<>(); tierBrokerMap.put("fast", "druid/fastBroker"); - tierBrokerMap.put("slow", "druid/broker"); + tierBrokerMap.put("fast", "druid/broker"); + tierBrokerMap.put("slow", "druid/slowBroker"); final TieredBrokerConfig tieredBrokerConfig = new TieredBrokerConfig() { @@ -74,8 +76,11 @@ public class JavaScriptTieredBrokerSelectorStrategyTest } }; - final Druids.TimeseriesQueryBuilder queryBuilder = Druids.newTimeseriesQueryBuilder().dataSource("test") + final TopNQueryBuilder queryBuilder = new TopNQueryBuilder().dataSource("test") .intervals("2014/2015") + .dimension("bigdim") + .metric("count") + .threshold(1) .aggregators( ImmutableList.of( new CountAggregatorFactory("count") @@ -83,7 +88,7 @@ public class JavaScriptTieredBrokerSelectorStrategyTest ); Assert.assertEquals( - Optional.of("druid/fastBroker"), + Optional.absent(), jsStrategy.getBrokerServiceName( tieredBrokerConfig, queryBuilder.build() @@ -92,13 +97,29 @@ public class JavaScriptTieredBrokerSelectorStrategyTest Assert.assertEquals( - Optional.of("druid/broker"), + Optional.absent(), jsStrategy.getBrokerServiceName( tieredBrokerConfig, Druids.newTimeBoundaryQueryBuilder().dataSource("test").bound("maxTime").build() ) ); + Assert.assertEquals( + Optional.of("druid/slowBroker"), + jsStrategy.getBrokerServiceName( + tieredBrokerConfig, + queryBuilder.aggregators( + ImmutableList.of( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("longSum", "a"), + new DoubleSumAggregatorFactory("doubleSum", "b") + ) + ).build() + ) + ); + + // in absence of tiers, expect the default + tierBrokerMap.clear(); Assert.assertEquals( Optional.of("druid/broker"), jsStrategy.getBrokerServiceName( diff --git a/services/pom.xml b/services/pom.xml index 0c45dc6c753..d7aedc56f41 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -17,102 +17,115 @@ ~ along with this program; if not, write to the Free Software ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - - 4.0.0 + + 4.0.0 + io.druid + druid-services + druid-services + druid-services + + io.druid - druid-services - druid-services - druid-services + druid + 0.7.0-SNAPSHOT + - - io.druid - druid - 0.7.0-SNAPSHOT - + + + io.airlift + airline + + + io.druid + druid-indexing-hadoop + ${project.parent.version} + + + io.druid + druid-indexing-service + ${project.parent.version} + + + io.druid + druid-server + ${project.parent.version} + + - - - io.airlift - airline - - - io.druid - druid-indexing-hadoop - ${project.parent.version} - - - io.druid - druid-indexing-service - ${project.parent.version} - - - io.druid - druid-server - ${project.parent.version} - - - - - - - maven-jar-plugin - - - - true - true - - - - - - org.apache.maven.plugins - maven-shade-plugin - 2.2 - - - package - - shade - - - - ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - - - - - - maven-assembly-plugin - - - distro-assembly - package - - assembly - - - - src/assembly/assembly.xml - - - - - - - + + + + maven-jar-plugin + + + + true + true + + + + + + org.apache.maven.plugins + maven-shade-plugin + 2.2 + + + package + + shade + + + + ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-assembly-plugin + + + distro-assembly + package + + assembly + + + + src/assembly/assembly.xml + + + + + + + diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index f0ceac08ac3..877a356103e 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -50,6 +50,7 @@ import io.druid.server.QueryResource; import io.druid.server.coordination.broker.DruidBroker; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.metrics.MetricsModule; +import io.druid.server.router.TieredBrokerConfig; import org.eclipse.jetty.server.Server; import java.util.List; @@ -78,7 +79,9 @@ public class CliBroker extends ServerRunnable @Override public void configure(Binder binder) { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/broker"); + binder.bindConstant().annotatedWith(Names.named("serviceName")).to( + TieredBrokerConfig.DEFAULT_BROKER_SERVICE_NAME + ); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8082); binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index b1f670350bb..f0f7a658425 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -29,18 +29,20 @@ import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.client.indexing.IndexingServiceClient; -import io.druid.metadata.MetadataRuleManager; -import io.druid.metadata.MetadataRuleManagerConfig; -import io.druid.metadata.MetadataRuleManagerProvider; -import io.druid.metadata.MetadataSegmentManager; -import io.druid.metadata.MetadataSegmentManagerConfig; -import io.druid.metadata.MetadataSegmentManagerProvider; import io.druid.guice.ConfigProvider; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; +import io.druid.metadata.MetadataRuleManager; +import io.druid.metadata.MetadataRuleManagerConfig; +import io.druid.metadata.MetadataRuleManagerProvider; +import io.druid.metadata.MetadataSegmentManager; +import io.druid.metadata.MetadataSegmentManagerConfig; +import io.druid.metadata.MetadataSegmentManagerProvider; +import io.druid.metadata.MetadataStorage; +import io.druid.metadata.MetadataStorageProvider; import io.druid.server.coordinator.DruidCoordinator; import io.druid.server.coordinator.DruidCoordinatorConfig; import io.druid.server.coordinator.LoadQueueTaskMaster; @@ -49,15 +51,16 @@ import io.druid.server.http.BackwardsCompatibleInfoResource; import io.druid.server.http.CoordinatorDynamicConfigsResource; import io.druid.server.http.CoordinatorRedirectInfo; import io.druid.server.http.CoordinatorResource; -import io.druid.server.http.MetadataResource; import io.druid.server.http.DatasourcesResource; import io.druid.server.http.InfoResource; +import io.druid.server.http.MetadataResource; import io.druid.server.http.RedirectFilter; import io.druid.server.http.RedirectInfo; import io.druid.server.http.RulesResource; import io.druid.server.http.ServersResource; import io.druid.server.http.TiersResource; import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.router.TieredBrokerConfig; import org.apache.curator.framework.CuratorFramework; import org.eclipse.jetty.server.Server; @@ -88,11 +91,15 @@ public class CliCoordinator extends ServerRunnable @Override public void configure(Binder binder) { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/coordinator"); + binder.bindConstant().annotatedWith(Names.named("serviceName")).to(TieredBrokerConfig.DEFAULT_COORDINATOR_SERVICE_NAME); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8081); ConfigProvider.bind(binder, DruidCoordinatorConfig.class); + binder.bind(MetadataStorage.class) + .toProvider(MetadataStorageProvider.class) + .in(ManageLifecycle.class); + JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); @@ -111,6 +118,7 @@ public class CliCoordinator extends ServerRunnable binder.bind(DruidCoordinator.class); + LifecycleModule.register(binder, MetadataStorage.class); LifecycleModule.register(binder, DruidCoordinator.class); binder.bind(JettyServerInitializer.class) diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index 6afd1fbfc1f..1cc05c016b1 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -83,7 +83,7 @@ public class CliHadoopIndexer implements Runnable final List extensionURLs = Lists.newArrayList(); for (String coordinate : extensionsConfig.getCoordinates()) { final ClassLoader coordinateLoader = Initialization.getClassLoaderForCoordinates( - aetherClient, coordinate + aetherClient, coordinate, extensionsConfig.getDefaultVersion() ); extensionURLs.addAll(Arrays.asList(((URLClassLoader) coordinateLoader).getURLs())); } @@ -96,7 +96,7 @@ public class CliHadoopIndexer implements Runnable // put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts for (String coordinate : allCoordinates) { final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates( - aetherClient, coordinate + aetherClient, coordinate, extensionsConfig.getDefaultVersion() ); driverURLs.addAll(Arrays.asList(((URLClassLoader) hadoopLoader).getURLs())); } diff --git a/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java index 007b56fc076..18a35da9743 100644 --- a/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java @@ -19,20 +19,31 @@ package io.druid.cli; +import com.google.api.client.repackaged.com.google.common.base.Preconditions; import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.google.api.client.util.Lists; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.TypeLiteral; +import com.google.inject.name.Names; import com.metamx.common.logger.Logger; import io.airlift.command.Arguments; import io.airlift.command.Command; import io.druid.indexer.HadoopDruidDetermineConfigurationJob; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.HadoopDruidIndexerJob; -import io.druid.indexer.HadoopIngestionSpec; import io.druid.indexer.JobHelper; import io.druid.indexer.Jobby; +import io.druid.indexer.MetadataStorageUpdaterJobHandler; +import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec; +import io.druid.metadata.MetadataStorageConnectorConfig; import java.io.File; import java.util.List; +import java.util.Properties; /** */ @@ -40,20 +51,58 @@ import java.util.List; name = "hadoop-indexer", description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/latest/Batch-ingestion.html for a description." ) -public class CliInternalHadoopIndexer implements Runnable +public class CliInternalHadoopIndexer extends GuiceRunnable { private static final Logger log = new Logger(CliHadoopIndexer.class); + @Arguments(description = "A JSON object or the path to a file that contains a JSON object", required = true) private String argumentSpec; + private HadoopDruidIndexerConfig config; + + public CliInternalHadoopIndexer() + { + super(log); + } + + @Override + protected List getModules() + { + return ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/internal-hadoop-indexer"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + + // bind metadata storage config based on HadoopIOConfig + MetadataStorageUpdaterJobSpec metadataSpec = getHadoopDruidIndexerConfig().getSchema() + .getIOConfig() + .getMetadataUpdateSpec(); + + binder.bind(new TypeLiteral>() {}) + .toInstance(metadataSpec); + } + } + ); + } + @Override public void run() { try { - HadoopDruidIndexerConfig config = getHadoopDruidIndexerConfig(); + Injector injector = makeInjector(); + + MetadataStorageUpdaterJobSpec metadataSpec = getHadoopDruidIndexerConfig().getSchema().getIOConfig().getMetadataUpdateSpec(); + // override metadata storage type based on HadoopIOConfig + Preconditions.checkNotNull(metadataSpec.getType(), "type in metadataUpdateSpec must not be null"); + injector.getInstance(Properties.class).setProperty("druid.metadata.storage.type", metadataSpec.getType()); + List jobs = Lists.newArrayList(); jobs.add(new HadoopDruidDetermineConfigurationJob(config)); - jobs.add(new HadoopDruidIndexerJob(config)); + jobs.add(new HadoopDruidIndexerJob(config, injector.getInstance(MetadataStorageUpdaterJobHandler.class))); JobHelper.runJobs(jobs, config); } @@ -64,16 +113,18 @@ public class CliInternalHadoopIndexer implements Runnable public HadoopDruidIndexerConfig getHadoopDruidIndexerConfig() { - try { - HadoopIngestionSpec spec; - if (argumentSpec.startsWith("{")) { - return HadoopDruidIndexerConfig.fromString(argumentSpec); - } else { - return HadoopDruidIndexerConfig.fromFile(new File(argumentSpec)); + if(config == null) { + try { + if (argumentSpec.startsWith("{")) { + config = HadoopDruidIndexerConfig.fromString(argumentSpec); + } else { + config = HadoopDruidIndexerConfig.fromFile(new File(argumentSpec)); + } + } + catch (Exception e) { + throw Throwables.propagate(e); } } - catch (Exception e) { - throw Throwables.propagate(e); - } + return config; } } diff --git a/services/src/main/java/io/druid/cli/PullDependencies.java b/services/src/main/java/io/druid/cli/PullDependencies.java index 41866d6d3a5..e48dd7658bc 100644 --- a/services/src/main/java/io/druid/cli/PullDependencies.java +++ b/services/src/main/java/io/druid/cli/PullDependencies.java @@ -67,7 +67,7 @@ public class PullDependencies implements Runnable try { final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig); for (final String coordinate : allCoordinates) { - Initialization.getClassLoaderForCoordinates(aetherClient, coordinate); + Initialization.getClassLoaderForCoordinates(aetherClient, coordinate, extensionsConfig.getDefaultVersion()); } } catch (Exception e) {