Add numeric StringComparator (#3270)

* Add numeric StringComparator

* Only use direct long comparison for numeric ordering in BoundFilter, add time filtering benchmark query

* Address PR comments, add multithreaded BoundDimFilter test

* Add comment on strlen tie handling

* Add timeseries interval filter benchmark

* Adjust docs

* Use jackson for StringComparator, address PR comments

* Add new TopNMetricSpec and SearchSortSpec with tests (WIP)

* More TopNMetricSpec and SearchSortSpec tests

* Fix NewSearchSortSpec serde

* Update docs for new DimensionTopNMetricSpec

* Delete NumericDimensionTopNMetricSpec

* Delete old SearchSortSpec

* Rename NewSearchSortSpec to SearchSortSpec

* Add TopN numeric comparator benchmark, address PR comments

* Refactor OrderByColumnSpec

* Add null checks to NumericComparator and String->BigDecimal conversion function

* Add more OrderByColumnSpec serde tests
This commit is contained in:
Jonathan Wei 2016-07-29 15:44:16 -07:00 committed by Gian Merlino
parent d04af6aee4
commit a6105cbb86
50 changed files with 1648 additions and 764 deletions

View File

@ -32,6 +32,7 @@ import com.metamx.collections.bitmap.RoaringBitmapFactory;
import com.metamx.collections.spatial.ImmutableRTree;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.column.BitmapIndex;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.ConciseBitmapSerdeFactory;
@ -75,7 +76,8 @@ public class BoundFilterBenchmark
true,
false,
false,
null
null,
StringComparators.LEXICOGRAPHIC
)
);
@ -87,7 +89,8 @@ public class BoundFilterBenchmark
false,
false,
false,
null
null,
StringComparators.LEXICOGRAPHIC
)
);
@ -99,7 +102,8 @@ public class BoundFilterBenchmark
false,
false,
false,
null
null,
StringComparators.LEXICOGRAPHIC
)
);
@ -111,7 +115,8 @@ public class BoundFilterBenchmark
true,
false,
true,
null
null,
StringComparators.ALPHANUMERIC
)
);
@ -123,7 +128,8 @@ public class BoundFilterBenchmark
false,
false,
true,
null
null,
StringComparators.ALPHANUMERIC
)
);
@ -135,7 +141,8 @@ public class BoundFilterBenchmark
false,
false,
true,
null
null,
StringComparators.ALPHANUMERIC
)
);

View File

@ -50,6 +50,7 @@ import io.druid.query.filter.DruidPredicateFactory;
import io.druid.query.filter.Filter;
import io.druid.query.filter.OrDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
import io.druid.segment.IndexIO;
@ -186,8 +187,9 @@ public class FilterPartitionBenchmark
String.valueOf(Long.MAX_VALUE),
true,
true,
true,
null
null,
null,
StringComparators.ALPHANUMERIC
));
long halfEnd = (interval.getEndMillis() + interval.getStartMillis()) / 2;
@ -197,8 +199,9 @@ public class FilterPartitionBenchmark
String.valueOf(halfEnd),
true,
true,
true,
null
null,
null,
StringComparators.ALPHANUMERIC
));
timeFilterAll = new BoundFilter(new BoundDimFilter(
@ -207,8 +210,9 @@ public class FilterPartitionBenchmark
String.valueOf(interval.getEndMillis()),
true,
true,
true,
null
null,
null,
StringComparators.ALPHANUMERIC
));
}

View File

@ -64,6 +64,7 @@ import io.druid.query.filter.OrDimFilter;
import io.druid.query.filter.RegexDimFilter;
import io.druid.query.filter.SearchQueryDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.query.spec.QuerySegmentSpec;
@ -185,7 +186,7 @@ public class FilteredAggregatorBenchmark
filter = new OrDimFilter(
Arrays.asList(
new BoundDimFilter("dimSequential", "-1", "-1", true, true, true, null),
new BoundDimFilter("dimSequential", "-1", "-1", true, true, null, null, StringComparators.ALPHANUMERIC),
new JavaScriptDimFilter("dimSequential", "function(x) { return false }", null, JavaScriptConfig.getDefault()),
new RegexDimFilter("dimSequential", "X", null),
new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null),

View File

@ -41,6 +41,7 @@ import io.druid.query.filter.JavaScriptDimFilter;
import io.druid.query.filter.OrDimFilter;
import io.druid.query.filter.RegexDimFilter;
import io.druid.query.filter.SearchQueryDimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
@ -164,7 +165,7 @@ public class IncrementalIndexReadBenchmark
{
DimFilter filter = new OrDimFilter(
Arrays.asList(
new BoundDimFilter("dimSequential", "-1", "-1", true, true, true, null),
new BoundDimFilter("dimSequential", "-1", "-1", true, true, null, null, StringComparators.ALPHANUMERIC),
new JavaScriptDimFilter("dimSequential", "function(x) { return false }", null, JavaScriptConfig.getDefault()),
new RegexDimFilter("dimSequential", "X", null),
new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null),

View File

@ -45,12 +45,15 @@ import io.druid.query.Result;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleMinAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.FilteredAggregatorFactory;
import io.druid.query.aggregation.LongMaxAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.query.spec.QuerySegmentSpec;
import io.druid.query.timeseries.TimeseriesQuery;
@ -64,11 +67,13 @@ import io.druid.segment.IndexMergerV9;
import io.druid.segment.IndexSpec;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.segment.incremental.OnheapIncrementalIndex;
import io.druid.segment.serde.ComplexMetrics;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -104,7 +109,7 @@ public class TimeseriesBenchmark
@Param({"750000"})
private int rowsPerSegment;
@Param({"basic.A"})
@Param({"basic.A", "basic.timeFilter", "basic.timeFilterAlphanumeric", "basic.timeFilterByInterval"})
private String schemaAndQuery;
private static final Logger log = new Logger(TimeseriesBenchmark.class);
@ -167,6 +172,64 @@ public class TimeseriesBenchmark
basicQueries.put("A", queryA);
}
{
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential");
BoundDimFilter timeFilter = new BoundDimFilter(Column.TIME_COLUMN_NAME, "200000", "300000", false, false, null, null,
StringComparators.NUMERIC);
queryAggs.add(new FilteredAggregatorFactory(lsaf, timeFilter));
TimeseriesQuery timeFilterQuery =
Druids.newTimeseriesQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.intervals(intervalSpec)
.aggregators(queryAggs)
.descending(false)
.build();
basicQueries.put("timeFilter", timeFilterQuery);
}
{
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential");
BoundDimFilter timeFilter = new BoundDimFilter(Column.TIME_COLUMN_NAME, "200000", "300000", false, false, null, null,
StringComparators.ALPHANUMERIC);
queryAggs.add(new FilteredAggregatorFactory(lsaf, timeFilter));
TimeseriesQuery timeFilterQuery =
Druids.newTimeseriesQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.intervals(intervalSpec)
.aggregators(queryAggs)
.descending(false)
.build();
basicQueries.put("timeFilterAlphanumeric", timeFilterQuery);
}
{
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(new Interval(200000, 300000)));
List<AggregatorFactory> queryAggs = new ArrayList<>();
LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential");
queryAggs.add(lsaf);
TimeseriesQuery timeFilterQuery =
Druids.newTimeseriesQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.intervals(intervalSpec)
.aggregators(queryAggs)
.descending(false)
.build();
basicQueries.put("timeFilterByInterval", timeFilterQuery);
}
SCHEMA_QUERY_MAP.put("basic", basicQueries);
}

View File

@ -50,8 +50,10 @@ import io.druid.query.aggregation.LongMaxAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.query.ordering.StringComparators;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.query.spec.QuerySegmentSpec;
import io.druid.query.topn.DimensionTopNMetricSpec;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNQueryBuilder;
import io.druid.query.topn.TopNQueryConfig;
@ -104,7 +106,7 @@ public class TopNBenchmark
@Param({"750000"})
private int rowsPerSegment;
@Param({"basic.A"})
@Param({"basic.A", "basic.numericSort", "basic.alphanumericSort"})
private String schemaAndQuery;
@Param({"10"})
@ -170,6 +172,38 @@ public class TopNBenchmark
basicQueries.put("A", queryBuilderA);
}
{ // basic.numericSort
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
TopNQueryBuilder queryBuilderA = new TopNQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.dimension("dimUniform")
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
.intervals(intervalSpec)
.aggregators(queryAggs);
basicQueries.put("numericSort", queryBuilderA);
}
{ // basic.alphanumericSort
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
TopNQueryBuilder queryBuilderA = new TopNQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.dimension("dimUniform")
.metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC))
.intervals(intervalSpec)
.aggregators(queryAggs);
basicQueries.put("alphanumericSort", queryBuilderA);
}
SCHEMA_QUERY_MAP.put("basic", basicQueries);
}

View File

@ -1,7 +1,7 @@
---
layout: doc_page
---
#Query Filters
# Query Filters
A filter is a JSON object indicating which rows of data should be included in the computation for a query. Its essentially the equivalent of the WHERE clause in SQL. Druid supports the following types of filters.
### Selector filter
@ -174,26 +174,33 @@ The IN filter supports the use of extraction functions, see [Filtering with Extr
### Bound filter
Bound filter can be used to filter by comparing dimension values to an upper value or/and a lower value.
By default Comparison is string based and **case sensitive**.
To use numeric comparison you can set `alphaNumeric` to `true`.
By default the bound filter is a not a strict inclusion `inputString <= upper && inputSting >= lower`.
The Bound filter can be used to filter by comparing dimension values to an upper value and/or a lower value.
|property|type|description|required?|
|--------|-----------|---------|---------|
|type|String|This should always be "bound".|yes|
|dimension|String|The dimension to filter on|yes|
|lower|String|The lower bound for the filter|no|
|upper|String|The upper bound for the filter|no|
|lowerStrict|Boolean|Perform strict comparison on the lower bound ("<" instead of "<=")|no, default: false|
|upperStrict|Boolean|Perform strict comparison on the upper bound (">" instead of ">=")|no, default: false|
|ordering|String|Specifies the sorting order to use when comparing values against the bound. Can be one of the following values: "lexicographic", "alphanumeric", "numeric", "strlen". See [Sorting Orders](./sorting-orders.html) for more details.|no, default: "lexicographic"|
|extractionFn|[Extraction function](#filtering-with-extraction-functions)| Extraction function to apply to the dimension|no|
The bound filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
The grammar for a bound filter is as follows:
The following bound filter expresses the condition `21 <= age <= 31`:
```json
{
"type": "bound",
"dimension": "age",
"lower": "21",
"upper": "31" ,
"alphaNumeric": true
"ordering": "numeric"
}
```
Equivalent to retain column if `21 <= age <= 31`
This filter expresses the condition `foo <= name <= hoo`, using the default lexicographic sorting order.
```json
{
"type": "bound",
@ -203,12 +210,7 @@ Equivalent to retain column if `21 <= age <= 31`
}
```
Equivalent to retain column if `foo <= name <= hoo`
In order to have a strict inclusion user can set `lowerStrict` or/and `upperStrict` to `true`
To have strict bounds:
Using strict bounds, this filter expresses the condition `21 < age < 31`
```json
{
"type": "bound",
@ -217,59 +219,31 @@ To have strict bounds:
"lowerStrict": true,
"upper": "31" ,
"upperStrict": true,
"alphaNumeric": true
}
```
Equivalent to retain column if `21 < age < 31`
To have strict upper bound:
```json
{
"type": "bound",
"dimension": "age",
"lower": "21",
"upper": "31" ,
"upperStrict": true,
"alphaNumeric": true
"ordering": "numeric"
}
```
Equivalent to retain column if `21 <= age < 31`
To compare to only an upper bound or lowe bound
The user can also specify a one-sided bound by omitting "upper" or "lower". This filter expresses `age < 31`.
```json
{
"type": "bound",
"dimension": "age",
"upper": "31" ,
"upperStrict": true,
"alphaNumeric": true
"ordering": "numeric"
}
```
Equivalent to retain column if `age < 31`
Likewise, this filter expresses `age >= 18`
```json
{
"type": "bound",
"dimension": "age",
"lower": "18" ,
"alphaNumeric": true
"ordering": "numeric"
}
```
Equivalent to retain column if ` 18 <= age`
For `alphaNumeric` comparator, in case of the dimension value includes none-digits you may expect **fuzzy matching**
If dimension value starts with a none digit, the filter will consider it out of range (`value < lowerBound` and `value > upperBound`)
If dimension value starts with digit and contains a none digits comparing will be done character wise.
For instance suppose lower bound is `100` and value is `10K` the filter will match (`100 < 10K` returns `true`) since `K` is greater than any digit
Now suppose that the lower bound is `110` the filter will not match (`110 < 10K` returns `false`)
#### Search Query Spec

View File

@ -24,8 +24,10 @@ OrderByColumnSpecs indicate how to do order by operations. Each order-by conditi
{
"dimension" : "<Any dimension or metric name>",
"direction" : <"ascending"|"descending">,
"dimensionOrder" : <"lexicographic(default)"|"alphanumeric"|"strlen">
"dimensionOrder" : <"lexicographic"(default)|"alphanumeric"|"strlen"|"numeric">
}
```
If only the dimension is provided (as a JSON string), the default order-by is ascending.
If only the dimension is provided (as a JSON string), the default order-by is ascending with lexicographic sorting.
See [Sorting Orders](./sorting-orders.html) for more information on the sorting orders specified by "dimensionOrder".

View File

@ -38,7 +38,7 @@ There are several main parts to a search query:
|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
|searchDimensions|The dimensions to run the search over. Excluding this means the search is run over all dimensions.|no|
|query|See [SearchQuerySpec](../querying/searchqueryspec.html).|yes|
|sort|An object specifying how the results of the search should be sorted. Possible types here are "lexicographic" (the default sort), "alphanumeric" and "strlen".|no|
|sort|An object specifying how the results of the search should be sorted.<br/>Possible types are "lexicographic" (the default sort), "alphanumeric", "strlen", and "numeric".<br/>See [Sorting Orders](./sorting-orders.html) for more details.|no|
|context|See [Context](../querying/query-context.html)|no|
The format of the result is:

View File

@ -0,0 +1,27 @@
---
layout: doc_page
---
# Sorting Orders
These sorting orders are used by the [TopNMetricSpec](./topnmetricspec.html), [SearchQuery](./searchquery.html), GroupByQuery's [LimitSpec](./limitspec.html), and [BoundFilter](./filters.html#bound-filter).
## Lexicographic
Sorts values by converting Strings to their UTF-8 byte array representations and comparing lexicgraphically, byte-by-byte.
## Alphanumeric
Suitable for strings with both numeric and non-numeric content, e.g.: "file12 sorts after file2"
See https://github.com/amjjd/java-alphanum for more details on how this ordering sorts values.
This ordering is not suitable for numbers with decimal points or negative numbers.
* For example, "1.3" precedes "1.15" in this ordering because "15" has more significant digits than "3".
* Negative numbers are sorted after positive numbers (because numeric characters precede the "-" in the negative numbers).
## Numeric
Sorts values as numbers, supports integers and floating point values. Negative values are supported.
This sorting order will try to parse all string values as numbers. Unparseable values are treated as nulls, and nulls precede numbers.
When comparing two unparseable values (e.g., "hello" and "world"), this ordering will sort by comparing the unparsed strings lexicographically.
## Strlen
Sorts values by the their string lengths. When there is a tie, this comparator falls back to using the String compareTo method.

View File

@ -28,38 +28,27 @@ The metric field can also be given as a JSON object. The grammar for dimension v
|type|this indicates a numeric sort|yes|
|metric|the actual metric field in which results will be sorted by|yes|
## Lexicographic TopNMetricSpec
## Dimension TopNMetricSpec
The grammar for dimension values sorted lexicographically is as follows:
This metric specification sorts TopN results by dimension value, using one of the sorting orders described here: [Sorting Orders](./sorting-orders.html)
|property|type|description|required?|
|--------|----|-----------|---------|
|type|String|this indicates a sort a dimension's values|yes, must be 'dimension'|
|ordering|String|Specifies the sorting order. Can be one of the following values: "lexicographic", "alphanumeric", "numeric", "strlen". See [Sorting Orders](./sorting-orders.html) for more details.|no, default: "lexicographic"|
|previousStop|String|the starting point of the sort. For example, if a previousStop value is 'b', all values before 'b' are discarded. This field can be used to paginate through all the dimension values.|no|
The following metricSpec uses lexicographic sorting.
```json
"metric": {
"type": "lexicographic",
"type": "dimension",
"ordering": "lexicographic",
"previousStop": "<previousStop_value>"
}
```
|property|description|required?|
|--------|-----------|---------|
|type|this indicates a lexicographic sort|yes|
|previousStop|the starting point of the lexicographic sort. For example, if a previousStop value is 'b', all values before 'b' are discarded. This field can be used to paginate through all the dimension values.|no|
## AlphaNumeric TopNMetricSpec
Sort dimension values in alpha-numeric order, i.e treating numbers differently from other characters in sorting the values.
The algorithm is based on [https://github.com/amjjd/java-alphanum](https://github.com/amjjd/java-alphanum).
```json
"metric": {
"type": "alphaNumeric",
"previousStop": "<previousStop_value>"
}
```
|property|description|required?|
|--------|-----------|---------|
|type|this indicates an alpha-numeric sort|yes|
|previousStop|the starting point of the alpha-numeric sort. For example, if a previousStop value is 'b', all values before 'b' are discarded. This field can be used to paginate through all the dimension values.|no|
Note that in earlier versions of Druid, the functionality provided by the DimensionTopNMetricSpec was handled by two separate spec types, Lexicographic and Alphanumeric (when only two sorting orders were supported). These spec types have been deprecated but are still usable.
## Inverted TopNMetricSpec
@ -75,4 +64,4 @@ Sort dimension values in inverted order, i.e inverts the order of the delegate m
|property|description|required?|
|--------|-----------|---------|
|type|this indicates an inverted sort|yes|
|metric|the delegate metric spec. |yes|
|metric|the delegate metric spec. |yes|

View File

@ -47,6 +47,7 @@ layout: toc
* [Joins](/docs/VERSION/querying/joins.html)
* [Multitenancy](/docs/VERSION/querying/multitenancy.html)
* [Caching](/docs/VERSION/querying/caching.html)
* [Sorting Orders](/docs/VERSION/querying/sorting-orders.html)
## Design
* [Overview](/docs/VERSION/design/design.html)

View File

@ -48,6 +48,7 @@ public class DefaultObjectMapper extends ObjectMapper
registerModule(new QueryGranularityModule());
registerModule(new AggregatorsModule());
registerModule(new SegmentsModule());
registerModule(new StringComparatorModule());
configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
configure(MapperFeature.AUTO_DETECT_GETTERS, false);

View File

@ -0,0 +1,45 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.jackson;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
public class StringComparatorModule extends SimpleModule
{
public StringComparatorModule()
{
super("StringComparatorModule");
setMixInAnnotation(StringComparator.class, StringComparatorMixin.class);
registerSubtypes(
new NamedType(StringComparators.LexicographicComparator.class, StringComparators.LEXICOGRAPHIC_NAME),
new NamedType(StringComparators.AlphanumericComparator.class, StringComparators.ALPHANUMERIC_NAME),
new NamedType(StringComparators.StrlenComparator.class, StringComparators.STRLEN_NAME),
new NamedType(StringComparators.NumericComparator.class, StringComparators.NUMERIC_NAME)
);
}
@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringComparator.class)
public static interface StringComparatorMixin {}
}

View File

@ -44,9 +44,9 @@ import io.druid.query.search.SearchResultValue;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import io.druid.query.search.search.FragmentSearchQuerySpec;
import io.druid.query.search.search.InsensitiveContainsSearchQuerySpec;
import io.druid.query.search.search.SearchSortSpec;
import io.druid.query.search.search.SearchQuery;
import io.druid.query.search.search.SearchQuerySpec;
import io.druid.query.search.search.SearchSortSpec;
import io.druid.query.select.PagingSpec;
import io.druid.query.select.SelectQuery;
import io.druid.query.spec.LegacySegmentSpec;

View File

@ -20,14 +20,20 @@
package io.druid.query.filter;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.base.Supplier;
import com.google.common.collect.BoundType;
import com.google.common.collect.Range;
import com.google.common.collect.RangeSet;
import com.google.common.collect.TreeRangeSet;
import com.google.common.primitives.Longs;
import com.metamx.common.StringUtils;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.filter.BoundFilter;
import java.nio.ByteBuffer;
@ -40,8 +46,9 @@ public class BoundDimFilter implements DimFilter
private final String lower;
private final boolean lowerStrict;
private final boolean upperStrict;
private final boolean alphaNumeric;
private final ExtractionFn extractionFn;
private final StringComparator ordering;
private final Supplier<DruidLongPredicate> longPredicateSupplier;
@JsonCreator
public BoundDimFilter(
@ -50,8 +57,9 @@ public class BoundDimFilter implements DimFilter
@JsonProperty("upper") String upper,
@JsonProperty("lowerStrict") Boolean lowerStrict,
@JsonProperty("upperStrict") Boolean upperStrict,
@JsonProperty("alphaNumeric") Boolean alphaNumeric,
@JsonProperty("extractionFn") ExtractionFn extractionFn
@Deprecated @JsonProperty("alphaNumeric") Boolean alphaNumeric,
@JsonProperty("extractionFn") ExtractionFn extractionFn,
@JsonProperty("ordering") StringComparator ordering
)
{
this.dimension = Preconditions.checkNotNull(dimension, "dimension can not be null");
@ -60,8 +68,27 @@ public class BoundDimFilter implements DimFilter
this.lower = lower;
this.lowerStrict = (lowerStrict == null) ? false : lowerStrict;
this.upperStrict = (upperStrict == null) ? false : upperStrict;
this.alphaNumeric = (alphaNumeric == null) ? false : alphaNumeric;
// For backwards compatibility, we retain the 'alphaNumeric' property. It will be used if the new 'ordering'
// property is missing. If both 'ordering' and 'alphaNumeric' are present, make sure they are consistent.
if (ordering == null) {
if (alphaNumeric == null || !alphaNumeric) {
this.ordering = StringComparators.LEXICOGRAPHIC;
} else {
this.ordering = StringComparators.ALPHANUMERIC;
}
} else {
this.ordering = ordering;
if (alphaNumeric != null) {
boolean orderingIsAlphanumeric = this.ordering.equals(StringComparators.ALPHANUMERIC);
Preconditions.checkState(
alphaNumeric == orderingIsAlphanumeric,
"mismatch between alphanumeric and ordering property"
);
}
}
this.extractionFn = extractionFn;
this.longPredicateSupplier = makeLongPredicateSupplier();
}
@JsonProperty
@ -94,12 +121,6 @@ public class BoundDimFilter implements DimFilter
return upperStrict;
}
@JsonProperty
public boolean isAlphaNumeric()
{
return alphaNumeric;
}
public boolean hasLowerBound()
{
return lower != null;
@ -116,6 +137,17 @@ public class BoundDimFilter implements DimFilter
return extractionFn;
}
@JsonProperty
public StringComparator getOrdering()
{
return ordering;
}
public Supplier<DruidLongPredicate> getLongPredicateSupplier()
{
return longPredicateSupplier;
}
@Override
public byte[] getCacheKey()
{
@ -131,22 +163,23 @@ public class BoundDimFilter implements DimFilter
byte lowerStrictByte = (this.isLowerStrict() == false) ? 0x0 : (byte) 1;
byte upperStrictByte = (this.isUpperStrict() == false) ? 0x0 : (byte) 1;
byte AlphaNumericByte = (this.isAlphaNumeric() == false) ? 0x0 : (byte) 1;
byte[] extractionFnBytes = extractionFn == null ? new byte[0] : extractionFn.getCacheKey();
byte[] orderingBytes = ordering.getCacheKey();
ByteBuffer boundCacheBuffer = ByteBuffer.allocate(
9
+ dimensionBytes.length
+ upperBytes.length
+ lowerBytes.length
+ extractionFnBytes.length
+ orderingBytes.length
);
boundCacheBuffer.put(DimFilterUtils.BOUND_CACHE_ID)
.put(boundType)
.put(upperStrictByte)
.put(lowerStrictByte)
.put(AlphaNumericByte)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(dimensionBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
@ -154,7 +187,9 @@ public class BoundDimFilter implements DimFilter
.put(DimFilterUtils.STRING_SEPARATOR)
.put(lowerBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(extractionFnBytes);
.put(extractionFnBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(orderingBytes);
return boundCacheBuffer.array();
}
@ -173,9 +208,12 @@ public class BoundDimFilter implements DimFilter
@Override
public RangeSet<String> getDimensionRangeSet(String dimension)
{
if (!Objects.equals(getDimension(), dimension) || getExtractionFn() != null || alphaNumeric) {
if (!(Objects.equals(getDimension(), dimension)
&& getExtractionFn() == null
&& ordering.equals(StringComparators.LEXICOGRAPHIC))) {
return null;
}
RangeSet<String> retSet = TreeRangeSet.create();
Range<String> range;
if (getLower() == null) {
@ -208,9 +246,6 @@ public class BoundDimFilter implements DimFilter
if (isUpperStrict() != that.isUpperStrict()) {
return false;
}
if (isAlphaNumeric() != that.isAlphaNumeric()) {
return false;
}
if (!getDimension().equals(that.getDimension())) {
return false;
}
@ -220,9 +255,12 @@ public class BoundDimFilter implements DimFilter
if (getLower() != null ? !getLower().equals(that.getLower()) : that.getLower() != null) {
return false;
}
return getExtractionFn() != null
? getExtractionFn().equals(that.getExtractionFn())
: that.getExtractionFn() == null;
if (getExtractionFn() != null
? !getExtractionFn().equals(that.getExtractionFn())
: that.getExtractionFn() != null) {
return false;
}
return getOrdering().equals(that.getOrdering());
}
@ -234,8 +272,90 @@ public class BoundDimFilter implements DimFilter
result = 31 * result + (getLower() != null ? getLower().hashCode() : 0);
result = 31 * result + (isLowerStrict() ? 1 : 0);
result = 31 * result + (isUpperStrict() ? 1 : 0);
result = 31 * result + (isAlphaNumeric() ? 1 : 0);
result = 31 * result + (getExtractionFn() != null ? getExtractionFn().hashCode() : 0);
result = 31 * result + getOrdering().hashCode();
return result;
}
private Supplier<DruidLongPredicate> makeLongPredicateSupplier()
{
return new Supplier<DruidLongPredicate>()
{
private final Object initLock = new Object();
private volatile boolean longsInitialized = false;
private volatile boolean hasLowerLongBoundVolatile;
private volatile boolean hasUpperLongBoundVolatile;
private volatile long lowerLongBoundVolatile;
private volatile long upperLongBoundVolatile;
@Override
public DruidLongPredicate get()
{
initLongData();
return new DruidLongPredicate()
{
private final boolean hasLowerLongBound = hasLowerLongBoundVolatile;
private final boolean hasUpperLongBound = hasUpperLongBoundVolatile;
private final long lowerLongBound = hasLowerLongBound ? lowerLongBoundVolatile : 0L;
private final long upperLongBound = hasUpperLongBound ? upperLongBoundVolatile : 0L;
@Override
public boolean applyLong(long input)
{
int lowerComparing = 1;
int upperComparing = 1;
if (hasLowerLongBound) {
lowerComparing = Long.compare(input, lowerLongBound);
}
if (hasUpperLongBound) {
upperComparing = Long.compare(upperLongBound, input);
}
if (lowerStrict && upperStrict) {
return ((lowerComparing > 0)) && (upperComparing > 0);
} else if (lowerStrict) {
return (lowerComparing > 0) && (upperComparing >= 0);
} else if (upperStrict) {
return (lowerComparing >= 0) && (upperComparing > 0);
}
return (lowerComparing >= 0) && (upperComparing >= 0);
}
};
}
private void initLongData()
{
if (longsInitialized) {
return;
}
synchronized (initLock) {
if (longsInitialized) {
return;
}
Long lowerLong = Longs.tryParse(Strings.nullToEmpty(lower));
if (hasLowerBound() && lowerLong != null) {
hasLowerLongBoundVolatile = true;
lowerLongBoundVolatile = lowerLong;
} else {
hasLowerLongBoundVolatile = false;
}
Long upperLong = Longs.tryParse(Strings.nullToEmpty(upper));
if (hasUpperBound() && upperLong != null) {
hasUpperLongBoundVolatile = true;
upperLongBoundVolatile = upperLong;
} else {
hasUpperLongBoundVolatile = false;
}
longsInitialized = true;
}
}
};
}
}

View File

@ -487,11 +487,6 @@ public class GroupByQuery extends BaseQuery<Row>
return addOrderByColumn(dimension, (OrderByColumnSpec.Direction) null);
}
public Builder addOrderByColumn(String dimension, String direction)
{
return addOrderByColumn(dimension, OrderByColumnSpec.determineDirection(direction));
}
public Builder addOrderByColumn(String dimension, OrderByColumnSpec.Direction direction)
{
return addOrderByColumn(new OrderByColumnSpec(dimension, direction));

View File

@ -40,7 +40,7 @@ import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.ordering.StringComparators;
import io.druid.query.ordering.StringComparators.StringComparator;
import io.druid.query.ordering.StringComparator;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;

View File

@ -21,16 +21,15 @@ package io.druid.query.groupby.orderby;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.metamx.common.IAE;
import com.metamx.common.ISE;
import com.metamx.common.StringUtils;
import io.druid.query.ordering.StringComparators;
import io.druid.query.ordering.StringComparators.StringComparator;
import io.druid.query.ordering.StringComparator;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@ -42,50 +41,74 @@ import java.util.Map;
*/
public class OrderByColumnSpec
{
public static enum Direction
public enum Direction
{
ASCENDING,
DESCENDING
DESCENDING;
/**
* Maintain a map of the enum values so that we can just do a lookup and get a null if it doesn't exist instead
* of an exception thrown.
*/
private static final Map<String, Direction> stupidEnumMap;
static {
final ImmutableMap.Builder<String, Direction> bob = ImmutableMap.builder();
for (Direction direction : Direction.values()) {
bob.put(direction.name(), direction);
}
stupidEnumMap = bob.build();
}
@JsonValue
@Override
public String toString()
{
return this.name().toLowerCase();
}
@JsonCreator
public static Direction fromString(String name)
{
final String upperName = name.toUpperCase();
Direction direction = stupidEnumMap.get(upperName);
if (direction == null) {
for (Direction dir : Direction.values()) {
if (dir.name().startsWith(upperName)) {
if (direction != null) {
throw new ISE("Ambiguous directions[%s] and [%s]", direction, dir);
}
direction = dir;
}
}
}
return direction;
}
}
public static final StringComparator DEFAULT_DIMENSION_ORDER = StringComparators.LEXICOGRAPHIC;
/**
* Maintain a map of the enum values so that we can just do a lookup and get a null if it doesn't exist instead
* of an exception thrown.
*/
private static final Map<String, Direction> stupidEnumMap;
static {
final ImmutableMap.Builder<String, Direction> bob = ImmutableMap.builder();
for (Direction direction : Direction.values()) {
bob.put(direction.toString(), direction);
}
stupidEnumMap = bob.build();
}
private final String dimension;
private final Direction direction;
private final StringComparator dimensionComparator;
@JsonCreator
public static OrderByColumnSpec create(Object obj)
public OrderByColumnSpec(
@JsonProperty("dimension") String dimension,
@JsonProperty("direction") Direction direction,
@JsonProperty("dimensionOrder") StringComparator dimensionComparator
)
{
Preconditions.checkNotNull(obj, "Cannot build an OrderByColumnSpec from a null object.");
this.dimension = dimension;
this.direction = direction == null ? Direction.ASCENDING : direction;
this.dimensionComparator = dimensionComparator == null ? DEFAULT_DIMENSION_ORDER : dimensionComparator;
}
if (obj instanceof String) {
return new OrderByColumnSpec(obj.toString(), null, null);
} else if (obj instanceof Map) {
final Map map = (Map) obj;
final String dimension = map.get("dimension").toString();
final Direction direction = determineDirection(map.get("direction"));
final StringComparator dimensionComparator = determinDimensionComparator(map.get("dimensionOrder"));
return new OrderByColumnSpec(dimension, direction, dimensionComparator);
} else {
throw new ISE("Cannot build an OrderByColumnSpec from a %s", obj.getClass());
}
@JsonCreator
public static OrderByColumnSpec fromString(String dimension)
{
return new OrderByColumnSpec(dimension, null, null);
}
public static OrderByColumnSpec asc(String dimension)
@ -136,75 +159,24 @@ public class OrderByColumnSpec
this(dimension, direction, null);
}
public OrderByColumnSpec(
String dimension,
Direction direction,
StringComparator dimensionComparator
)
{
this.dimension = dimension;
this.direction = direction == null ? Direction.ASCENDING : direction;
this.dimensionComparator = dimensionComparator == null ? DEFAULT_DIMENSION_ORDER : dimensionComparator;
}
@JsonProperty
@JsonProperty("dimension")
public String getDimension()
{
return dimension;
}
@JsonProperty
@JsonProperty("direction")
public Direction getDirection()
{
return direction;
}
@JsonProperty
@JsonProperty("dimensionOrder")
public StringComparator getDimensionComparator()
{
return dimensionComparator;
}
public static Direction determineDirection(Object directionObj)
{
if (directionObj == null) {
return null;
}
String directionString = directionObj.toString();
Direction direction = stupidEnumMap.get(directionString);
if (direction == null) {
final String lowerDimension = directionString.toLowerCase();
for (Direction dir : Direction.values()) {
if (dir.toString().toLowerCase().startsWith(lowerDimension)) {
if (direction != null) {
throw new ISE("Ambiguous directions[%s] and [%s]", direction, dir);
}
direction = dir;
}
}
}
if (direction == null) {
throw new IAE("Unknown direction[%s]", directionString);
}
return direction;
}
private static StringComparator determinDimensionComparator(Object dimensionOrderObj)
{
if (dimensionOrderObj == null) {
return DEFAULT_DIMENSION_ORDER;
}
String dimensionOrderString = dimensionOrderObj.toString().toLowerCase();
return StringComparators.makeComparator(dimensionOrderString);
}
@Override
public boolean equals(Object o)
{

View File

@ -17,29 +17,31 @@
* under the License.
*/
package io.druid.query.search;
package io.druid.query.ordering;
import io.druid.query.search.search.SearchHit;
import io.druid.query.search.search.SearchSortSpec;
import io.druid.query.search.search.StrlenSearchSortSpec;
import org.junit.Assert;
import org.junit.Test;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.metamx.common.IAE;
/**
*/
public class StrlenSearchSortSpecTest
import java.util.Comparator;
public abstract class StringComparator implements Comparator<String>
{
@Test
public void testComparator()
@JsonCreator
public static StringComparator fromString(String type)
{
SearchSortSpec spec = new StrlenSearchSortSpec();
SearchHit hit1 = new SearchHit("test", "a");
SearchHit hit2 = new SearchHit("test", "apple");
SearchHit hit3 = new SearchHit("test", "elppa");
Assert.assertTrue(spec.getComparator().compare(hit2, hit3) < 0);
Assert.assertTrue(spec.getComparator().compare(hit2, hit1) > 0);
Assert.assertTrue(spec.getComparator().compare(hit1, hit3) < 0);
switch (type) {
case StringComparators.LEXICOGRAPHIC_NAME:
return StringComparators.LEXICOGRAPHIC;
case StringComparators.ALPHANUMERIC_NAME:
return StringComparators.ALPHANUMERIC;
case StringComparators.STRLEN_NAME:
return StringComparators.STRLEN;
case StringComparators.NUMERIC_NAME:
return StringComparators.NUMERIC;
default:
throw new IAE("Unknown string comparator[%s]", type);
}
}
public abstract byte[] getCacheKey();
}

View File

@ -19,40 +19,33 @@
package io.druid.query.ordering;
import java.math.BigDecimal;
import java.util.Comparator;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.annotation.JsonTypeInfo.As;
import com.fasterxml.jackson.annotation.JsonTypeInfo.Id;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Ints;
import com.google.common.primitives.Longs;
import com.google.common.primitives.UnsignedBytes;
import com.metamx.common.IAE;
import com.metamx.common.StringUtils;
public class StringComparators
{
public static final String LEXICOGRAPHIC_NAME = "lexicographic";
public static final String ALPHANUMERIC_NAME = "alphanumeric";
public static final String NUMERIC_NAME = "numeric";
public static final String STRLEN_NAME = "strlen";
public static final LexicographicComparator LEXICOGRAPHIC = new LexicographicComparator();
public static final AlphanumericComparator ALPHANUMERIC = new AlphanumericComparator();
public static final StrlenComparator STRLEN = new StrlenComparator();
public static final StringComparator LEXICOGRAPHIC = new LexicographicComparator();
public static final StringComparator ALPHANUMERIC = new AlphanumericComparator();
public static final StringComparator NUMERIC = new NumericComparator();
public static final StringComparator STRLEN = new StrlenComparator();
@JsonTypeInfo(use=Id.NAME, include=As.PROPERTY, property="type", defaultImpl = LexicographicComparator.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = StringComparators.LEXICOGRAPHIC_NAME, value = LexicographicComparator.class),
@JsonSubTypes.Type(name = StringComparators.ALPHANUMERIC_NAME, value = AlphanumericComparator.class),
@JsonSubTypes.Type(name = StringComparators.STRLEN_NAME, value = StrlenComparator.class)
})
public static interface StringComparator extends Comparator<String>
{
}
public static class LexicographicComparator implements StringComparator
public static final int LEXICOGRAPHIC_CACHE_ID = 0x01;
public static final int ALPHANUMERIC_CACHE_ID = 0x02;
public static final int NUMERIC_CACHE_ID = 0x03;
public static final int STRLEN_CACHE_ID = 0x04;
public static class LexicographicComparator extends StringComparator
{
private static final Ordering<String> ORDERING = Ordering.from(new Comparator<String>()
{
@ -74,7 +67,7 @@ public class StringComparators
return ORDERING.compare(s, s2);
}
@Override
public boolean equals(Object o)
{
@ -87,15 +80,21 @@ public class StringComparators
return true;
}
@Override
public String toString()
{
return StringComparators.LEXICOGRAPHIC_NAME;
}
@Override
public byte[] getCacheKey()
{
return new byte[]{(byte) LEXICOGRAPHIC_CACHE_ID};
}
}
public static class AlphanumericComparator implements StringComparator
public static class AlphanumericComparator extends StringComparator
{
// This code is based on https://github.com/amjjd/java-alphanum, see
// NOTICE file for more information
@ -299,9 +298,15 @@ public class StringComparators
{
return StringComparators.ALPHANUMERIC_NAME;
}
@Override
public byte[] getCacheKey()
{
return new byte[]{(byte) ALPHANUMERIC_CACHE_ID};
}
}
public static class StrlenComparator implements StringComparator
public static class StrlenComparator extends StringComparator
{
private static final Ordering<String> ORDERING = Ordering.from(new Comparator<String>()
{
@ -340,18 +345,93 @@ public class StringComparators
{
return StringComparators.STRLEN_NAME;
}
@Override
public byte[] getCacheKey()
{
return new byte[]{(byte) STRLEN_CACHE_ID};
}
}
public static StringComparator makeComparator(String type)
private static BigDecimal convertStringToBigDecimal(String input) {
if (input == null) {
return null;
}
// treat unparseable Strings as nulls
BigDecimal bd = null;
try {
bd = new BigDecimal(input);
} catch (NumberFormatException ex) {
}
return bd;
}
public static class NumericComparator extends StringComparator
{
if (type.equals(StringComparators.LEXICOGRAPHIC_NAME)) {
return LEXICOGRAPHIC;
} else if (type.equals(StringComparators.ALPHANUMERIC_NAME)) {
return ALPHANUMERIC;
} else if (type.equals(StringComparators.STRLEN_NAME)) {
return STRLEN;
} else {
throw new IAE("Unknown string comparator[%s]", type);
@Override
public int compare(String o1, String o2)
{
// return if o1 and o2 are the same object
if (o1 == o2) {
return 0;
}
// we know o1 != o2
if (o1 == null) {
return -1;
}
if (o2 == null) {
return 1;
}
// Creating a BigDecimal from a String is expensive (involves copying the String into a char[])
// Converting the String to a Long first is faster.
// We optimize here with the assumption that integer values are more common than floating point.
Long long1 = Longs.tryParse(o1);
Long long2 = Longs.tryParse(o2);
final BigDecimal bd1 = long1 == null ? convertStringToBigDecimal(o1) : new BigDecimal(long1);
final BigDecimal bd2 = long2 == null ? convertStringToBigDecimal(o2) : new BigDecimal(long2);
if (bd1 != null && bd2 != null) {
return bd1.compareTo(bd2);
}
if (bd1 == null && bd2 == null) {
// both Strings are unparseable, just compare lexicographically to have a well-defined ordering
return StringComparators.LEXICOGRAPHIC.compare(o1, o2);
}
if (bd1 == null) {
return -1;
} else {
return 1;
}
}
@Override
public String toString()
{
return StringComparators.NUMERIC_NAME;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
return true;
}
@Override
public byte[] getCacheKey()
{
return new byte[]{(byte) NUMERIC_CACHE_ID};
}
}
}

View File

@ -25,8 +25,8 @@ import com.metamx.common.guava.nary.BinaryFn;
import io.druid.granularity.AllGranularity;
import io.druid.granularity.QueryGranularity;
import io.druid.query.Result;
import io.druid.query.search.search.SearchHit;
import io.druid.query.search.search.SearchSortSpec;
import io.druid.query.search.search.SearchHit;
import org.joda.time.DateTime;
import java.util.Arrays;

View File

@ -1,78 +0,0 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.search.search;
import com.fasterxml.jackson.annotation.JsonCreator;
import io.druid.query.ordering.StringComparators;
import java.util.Comparator;
/**
*/
public class AlphanumericSearchSortSpec implements SearchSortSpec
{
@JsonCreator
public AlphanumericSearchSortSpec(
)
{
}
@Override
public Comparator<SearchHit> getComparator()
{
return new Comparator<SearchHit>()
{
@Override
public int compare(SearchHit searchHit1, SearchHit searchHit2)
{
int retVal = StringComparators.ALPHANUMERIC.compare(
searchHit1.getValue(), searchHit2.getValue());
if (retVal == 0) {
retVal = StringComparators.LEXICOGRAPHIC.compare(
searchHit1.getDimension(), searchHit2.getDimension());
}
return retVal;
}
};
}
public String toString()
{
return "alphanumericSort";
}
@Override
public boolean equals(Object other) {
return this == other || other instanceof AlphanumericSearchSortSpec;
}
@Override
public int hashCode()
{
return 0;
}
@Override
public byte[] getCacheKey()
{
return toString().getBytes();
}
}

View File

@ -1,79 +0,0 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.search.search;
import com.fasterxml.jackson.annotation.JsonCreator;
import io.druid.query.ordering.StringComparators;
import java.util.Comparator;
/**
*/
public class LexicographicSearchSortSpec implements SearchSortSpec
{
@JsonCreator
public LexicographicSearchSortSpec(
)
{
}
@Override
public Comparator<SearchHit> getComparator()
{
return new Comparator<SearchHit>()
{
@Override
public int compare(SearchHit searchHit, SearchHit searchHit1)
{
int retVal = StringComparators.LEXICOGRAPHIC.compare(
searchHit.getValue(), searchHit1.getValue());
if (retVal == 0) {
retVal = StringComparators.LEXICOGRAPHIC.compare(
searchHit.getDimension(), searchHit1.getDimension());
}
return retVal;
}
};
}
@Override
public byte[] getCacheKey()
{
return toString().getBytes();
}
public String toString()
{
return "lexicographicSort";
}
@Override
public boolean equals(Object other) {
return this == other || other instanceof LexicographicSearchSortSpec;
}
@Override
public int hashCode()
{
return 0;
}
}

View File

@ -30,6 +30,7 @@ import io.druid.query.Query;
import io.druid.query.Result;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.filter.DimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.SearchResultValue;
import io.druid.query.spec.QuerySegmentSpec;
@ -40,6 +41,8 @@ import java.util.Map;
*/
public class SearchQuery extends BaseQuery<Result<SearchResultValue>>
{
private static final SearchSortSpec DEFAULT_SORT_SPEC = new SearchSortSpec(StringComparators.LEXICOGRAPHIC);
private final DimFilter dimFilter;
private final SearchSortSpec sortSpec;
private final QueryGranularity granularity;
@ -62,7 +65,7 @@ public class SearchQuery extends BaseQuery<Result<SearchResultValue>>
{
super(dataSource, querySegmentSpec, false, context);
this.dimFilter = dimFilter;
this.sortSpec = sortSpec == null ? new LexicographicSearchSortSpec() : sortSpec;
this.sortSpec = sortSpec == null ? DEFAULT_SORT_SPEC : sortSpec;
this.granularity = granularity == null ? QueryGranularities.ALL : granularity;
this.limit = (limit == 0) ? 1000 : limit;
this.dimensions = dimensions;

View File

@ -19,22 +19,82 @@
package io.druid.query.search.search;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.common.StringUtils;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
import java.util.Comparator;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LexicographicSearchSortSpec.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "lexicographic", value = LexicographicSearchSortSpec.class),
@JsonSubTypes.Type(name = "alphanumeric", value = AlphanumericSearchSortSpec.class),
@JsonSubTypes.Type(name = "strlen", value = StrlenSearchSortSpec.class)
})
public interface SearchSortSpec
public class SearchSortSpec
{
Comparator<SearchHit> getComparator();
public static final StringComparator DEFAULT_ORDERING = StringComparators.LEXICOGRAPHIC;
byte[] getCacheKey();
private final StringComparator ordering;
@JsonCreator
public SearchSortSpec(
@JsonProperty("type") StringComparator ordering
)
{
this.ordering = ordering == null ? DEFAULT_ORDERING : ordering;
}
@JsonProperty("type")
public StringComparator getOrdering()
{
return ordering;
}
public Comparator<SearchHit> getComparator()
{
return new Comparator<SearchHit>()
{
@Override
public int compare(SearchHit searchHit, SearchHit searchHit1)
{
int retVal = ordering.compare(
searchHit.getValue(), searchHit1.getValue());
if (retVal == 0) {
retVal = StringComparators.LEXICOGRAPHIC.compare(
searchHit.getDimension(), searchHit1.getDimension());
}
return retVal;
}
};
}
public byte[] getCacheKey()
{
return ordering.getCacheKey();
}
public String toString()
{
return String.format("%sSort", ordering.toString());
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SearchSortSpec that = (SearchSortSpec) o;
return ordering.equals(that.ordering);
}
@Override
public int hashCode()
{
return ordering.hashCode();
}
}

View File

@ -1,72 +0,0 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.search.search;
import io.druid.query.ordering.StringComparators;
import java.util.Comparator;
/**
*/
public class StrlenSearchSortSpec implements SearchSortSpec
{
public StrlenSearchSortSpec()
{
}
@Override
public Comparator<SearchHit> getComparator()
{
return new Comparator<SearchHit>() {
@Override
public int compare(SearchHit s, SearchHit s1)
{
int res = StringComparators.STRLEN.compare(s.getValue(), s1.getValue());
if (res == 0) {
res = StringComparators.LEXICOGRAPHIC.compare(
s.getDimension(), s1.getDimension());
}
return res;
}
};
}
@Override
public byte[] getCacheKey()
{
return toString().getBytes();
}
public String toString()
{
return "stringLengthSort";
}
@Override
public boolean equals(Object other) {
return this == other || other instanceof StrlenSearchSortSpec;
}
@Override
public int hashCode()
{
return 0;
}
}

View File

@ -0,0 +1,182 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.topn;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
import org.joda.time.DateTime;
import java.nio.ByteBuffer;
import java.util.Comparator;
import java.util.List;
/**
*/
public class DimensionTopNMetricSpec implements TopNMetricSpec
{
private static final StringComparator DEFAULT_ORDERING = StringComparators.LEXICOGRAPHIC;
private static final byte STRING_SEPARATOR = (byte) 0xFF;
private static final byte CACHE_TYPE_ID = 0x4;
private final String previousStop;
private final StringComparator ordering;
@JsonCreator
public DimensionTopNMetricSpec(
@JsonProperty("previousStop") String previousStop,
@JsonProperty("ordering") StringComparator ordering
)
{
this.previousStop = previousStop;
this.ordering = ordering == null ? DEFAULT_ORDERING : ordering;
}
@Override
public void verifyPreconditions(List<AggregatorFactory> aggregatorSpecs, List<PostAggregator> postAggregatorSpecs)
{
}
@JsonProperty
public String getPreviousStop()
{
return previousStop;
}
@JsonProperty
public StringComparator getOrdering()
{
return ordering;
}
@Override
public Comparator getComparator(List<AggregatorFactory> aggregatorSpecs, List<PostAggregator> postAggregatorSpecs)
{
return ordering;
}
@Override
public TopNResultBuilder getResultBuilder(
DateTime timestamp,
DimensionSpec dimSpec,
int threshold,
Comparator comparator,
List<AggregatorFactory> aggFactories,
List<PostAggregator> postAggs
)
{
return new TopNLexicographicResultBuilder(
timestamp,
dimSpec,
threshold,
previousStop,
comparator,
aggFactories
);
}
@Override
public byte[] getCacheKey()
{
byte[] previousStopBytes = previousStop == null ? new byte[]{} : StringUtils.toUtf8(previousStop);
byte[] orderingBytes = ordering.getCacheKey();
int totalLen = 2 + previousStopBytes.length + orderingBytes.length;
return ByteBuffer.allocate(totalLen)
.put(CACHE_TYPE_ID)
.put(previousStopBytes)
.put(STRING_SEPARATOR)
.put(orderingBytes)
.array();
}
@Override
public <T> TopNMetricSpecBuilder<T> configureOptimizer(TopNMetricSpecBuilder<T> builder)
{
if (ordering.equals(StringComparators.LEXICOGRAPHIC)) {
builder.skipTo(previousStop);
builder.ignoreAfterThreshold();
}
return builder;
}
@Override
public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector)
{
selector.setAggregateAllMetrics(true);
}
@Override
public String getMetricName(DimensionSpec dimSpec)
{
return dimSpec.getOutputName();
}
@Override
public boolean canBeOptimizedUnordered()
{
return false;
}
@Override
public String toString()
{
return "DimensionTopNMetricSpec{" +
"previousStop='" + previousStop + '\'' +
"ordering='" + ordering + '\'' +
'}';
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DimensionTopNMetricSpec that = (DimensionTopNMetricSpec) o;
if (getPreviousStop() != null
? !getPreviousStop().equals(that.getPreviousStop())
: that.getPreviousStop() != null) {
return false;
}
return getOrdering().equals(that.getOrdering());
}
@Override
public int hashCode()
{
int result = getPreviousStop() != null ? getPreviousStop().hashCode() : 0;
result = 31 * result + getOrdering().hashCode();
return result;
}
}

View File

@ -36,7 +36,8 @@ import java.util.List;
@JsonSubTypes.Type(name = "numeric", value = NumericTopNMetricSpec.class),
@JsonSubTypes.Type(name = "lexicographic", value = LexicographicTopNMetricSpec.class),
@JsonSubTypes.Type(name = "alphaNumeric", value = AlphaNumericTopNMetricSpec.class),
@JsonSubTypes.Type(name = "inverted", value = InvertedTopNMetricSpec.class)
@JsonSubTypes.Type(name = "inverted", value = InvertedTopNMetricSpec.class),
@JsonSubTypes.Type(name = "dimension", value = DimensionTopNMetricSpec.class),
})
public interface TopNMetricSpec
{

View File

@ -20,6 +20,9 @@
package io.druid.segment.filter;
import com.google.common.base.Predicate;
import com.google.common.base.Strings;
import com.google.common.base.Supplier;
import com.google.common.primitives.Longs;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BitmapIndexSelector;
@ -41,19 +44,20 @@ public class BoundFilter implements Filter
private final Comparator<String> comparator;
private final ExtractionFn extractionFn;
private final Supplier<DruidLongPredicate> longPredicateSupplier;
public BoundFilter(final BoundDimFilter boundDimFilter)
{
this.boundDimFilter = boundDimFilter;
this.comparator = boundDimFilter.isAlphaNumeric()
? StringComparators.ALPHANUMERIC
: StringComparators.LEXICOGRAPHIC;
this.comparator = boundDimFilter.getOrdering();
this.extractionFn = boundDimFilter.getExtractionFn();
this.longPredicateSupplier = boundDimFilter.getLongPredicateSupplier();
}
@Override
public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
{
if (boundDimFilter.isAlphaNumeric() || extractionFn != null) {
if (!boundDimFilter.getOrdering().equals(StringComparators.LEXICOGRAPHIC_NAME) || extractionFn != null) {
return Filters.matchPredicate(
boundDimFilter.getDimension(),
selector,
@ -151,39 +155,57 @@ public class BoundFilter implements Filter
@Override
public Predicate<String> makeStringPredicate()
{
return new Predicate<String>()
{
@Override
public boolean apply(String input)
if (extractionFn != null) {
return new Predicate<String>()
{
return doesMatch(input);
}
};
@Override
public boolean apply(String input)
{
return doesMatch(extractionFn.apply(input));
}
};
} else {
return new Predicate<String>()
{
@Override
public boolean apply(String input)
{
return doesMatch(input);
}
};
}
}
@Override
public DruidLongPredicate makeLongPredicate()
{
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
if (extractionFn != null) {
return new DruidLongPredicate()
{
// When BoundFilter has a 'numeric' comparator (see https://github.com/druid-io/druid/issues/2989)
// this should be optimized to compare on longs instead of using string conversion.
return doesMatch(String.valueOf(input));
}
};
@Override
public boolean applyLong(long input)
{
return doesMatch(extractionFn.apply(input));
}
};
} else if (boundDimFilter.getOrdering().equals(StringComparators.NUMERIC_NAME)){
return longPredicateSupplier.get();
} else {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return doesMatch(String.valueOf(input));
}
};
}
}
};
}
private boolean doesMatch(String input)
{
if (extractionFn != null) {
input = extractionFn.apply(input);
}
if (input == null) {
return (!boundDimFilter.hasLowerBound()
|| (boundDimFilter.getLower().isEmpty() && !boundDimFilter.isLowerStrict())) // lower bound allows null

View File

@ -34,6 +34,7 @@ import io.druid.query.filter.OrDimFilter;
import io.druid.query.filter.RegexDimFilter;
import io.druid.query.filter.SearchQueryDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.DimensionSelector;
@ -257,7 +258,7 @@ public class FilteredAggregatorTest
factory = new FilteredAggregatorFactory(
new DoubleSumAggregatorFactory("billy", "value"),
new BoundDimFilter("dim", "a", "a", false, false, true, null)
new BoundDimFilter("dim", "a", "a", false, false, true, null, StringComparators.ALPHANUMERIC)
);
selector = new TestFloatColumnSelector(values);
validateFilteredAggs(factory, values, selector);
@ -308,10 +309,12 @@ public class FilteredAggregatorTest
);
selector = new TestFloatColumnSelector(values);
validateFilteredAggs(factory, values, selector);
factory = new FilteredAggregatorFactory(
new DoubleSumAggregatorFactory("billy", "value"),
new BoundDimFilter("dim", "aAARDVARK", "aAARDVARK", false, false, true, extractionFn)
new BoundDimFilter("dim", "aAARDVARK", "aAARDVARK", false, false, true, extractionFn,
StringComparators.ALPHANUMERIC
)
);
selector = new TestFloatColumnSelector(values);
validateFilteredAggs(factory, values, selector);

View File

@ -29,6 +29,7 @@ import io.druid.guice.GuiceInjectors;
import io.druid.guice.annotations.Json;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.RegexDimExtractionFn;
import io.druid.query.ordering.StringComparators;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -50,15 +51,24 @@ public class BoundDimFilterTest
public static Iterable<Object[]> constructorFeeder(){
return ImmutableList.of(
new Object[]{new BoundDimFilter("dimension", "12", "15", null, null, null, null)},
new Object[]{new BoundDimFilter("dimension", "12", "15", null, true, false, null)},
new Object[]{new BoundDimFilter("dimension", "12", "15", null, null, true, null)},
new Object[]{new BoundDimFilter("dimension", null, "15", null, true, true, null)},
new Object[]{new BoundDimFilter("dimension", "12", "15", true, null, null, null)},
new Object[]{new BoundDimFilter("dimension", "12", null, true, null, true, null)},
new Object[]{new BoundDimFilter("dimension", "12", "15", true, true, true, null)},
new Object[]{new BoundDimFilter("dimension", "12", "15", true, true, false, null)},
new Object[]{new BoundDimFilter("dimension", null, "15", null, true, true, extractionFn)}
new Object[]{new BoundDimFilter("dimension", "12", "15", null, null, null, null,
StringComparators.LEXICOGRAPHIC)},
new Object[]{new BoundDimFilter("dimension", "12", "15", null, true, false, null,
StringComparators.LEXICOGRAPHIC)},
new Object[]{new BoundDimFilter("dimension", "12", "15", null, null, true, null,
StringComparators.ALPHANUMERIC)},
new Object[]{new BoundDimFilter("dimension", null, "15", null, true, true, null,
StringComparators.ALPHANUMERIC)},
new Object[]{new BoundDimFilter("dimension", "12", "15", true, null, null, null,
StringComparators.LEXICOGRAPHIC)},
new Object[]{new BoundDimFilter("dimension", "12", null, true, null, true, null,
StringComparators.ALPHANUMERIC)},
new Object[]{new BoundDimFilter("dimension", "12", "15", true, true, true, null,
StringComparators.ALPHANUMERIC)},
new Object[]{new BoundDimFilter("dimension", "12", "15", true, true, false, null,
StringComparators.LEXICOGRAPHIC)},
new Object[]{new BoundDimFilter("dimension", null, "15", null, true, true, extractionFn,
StringComparators.ALPHANUMERIC)}
);
}
@ -75,14 +85,14 @@ public class BoundDimFilterTest
@Test
public void testGetCacheKey()
{
BoundDimFilter boundDimFilter = new BoundDimFilter("dimension", "12", "15", null, null, true, null);
BoundDimFilter boundDimFilterCopy = new BoundDimFilter("dimension", "12", "15", false, false, true, null);
BoundDimFilter boundDimFilter = new BoundDimFilter("dimension", "12", "15", null, null, true, null, StringComparators.ALPHANUMERIC);
BoundDimFilter boundDimFilterCopy = new BoundDimFilter("dimension", "12", "15", false, false, true, null, StringComparators.ALPHANUMERIC);
Assert.assertArrayEquals(boundDimFilter.getCacheKey(), boundDimFilterCopy.getCacheKey());
BoundDimFilter anotherBoundDimFilter = new BoundDimFilter("dimension", "12", "15", true, null, false, null);
BoundDimFilter anotherBoundDimFilter = new BoundDimFilter("dimension", "12", "15", true, null, false, null, StringComparators.LEXICOGRAPHIC);
Assert.assertFalse(Arrays.equals(anotherBoundDimFilter.getCacheKey(), boundDimFilter.getCacheKey()));
BoundDimFilter boundDimFilterWithExtract = new BoundDimFilter("dimension", "12", "15", null, null, true, extractionFn);
BoundDimFilter boundDimFilterWithExtractCopy = new BoundDimFilter("dimension", "12", "15", false, false, true, extractionFn);
BoundDimFilter boundDimFilterWithExtract = new BoundDimFilter("dimension", "12", "15", null, null, true, extractionFn, StringComparators.ALPHANUMERIC);
BoundDimFilter boundDimFilterWithExtractCopy = new BoundDimFilter("dimension", "12", "15", false, false, true, extractionFn, StringComparators.ALPHANUMERIC);
Assert.assertFalse(Arrays.equals(boundDimFilter.getCacheKey(), boundDimFilterWithExtract.getCacheKey()));
Assert.assertArrayEquals(boundDimFilterWithExtract.getCacheKey(), boundDimFilterWithExtractCopy.getCacheKey());
}
@ -90,8 +100,8 @@ public class BoundDimFilterTest
@Test
public void testHashCode()
{
BoundDimFilter boundDimFilter = new BoundDimFilter("dimension", "12", "15", null, null, true, null);
BoundDimFilter boundDimFilterWithExtract = new BoundDimFilter("dimension", "12", "15", null, null, true, extractionFn);
BoundDimFilter boundDimFilter = new BoundDimFilter("dimension", "12", "15", null, null, true, null, StringComparators.ALPHANUMERIC);
BoundDimFilter boundDimFilterWithExtract = new BoundDimFilter("dimension", "12", "15", null, null, true, extractionFn, StringComparators.ALPHANUMERIC);
Assert.assertNotEquals(boundDimFilter.hashCode(), boundDimFilterWithExtract.hashCode());
}

View File

@ -25,6 +25,7 @@ import com.google.common.collect.Range;
import com.google.common.collect.RangeSet;
import io.druid.js.JavaScriptConfig;
import io.druid.query.extraction.IdentityExtractionFn;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import org.junit.Assert;
import org.junit.Test;
@ -43,10 +44,18 @@ public class GetDimensionRangeSetTest
private final DimFilter in1 = new InDimFilter("dim1", ImmutableList.of("testing", "this", "filter", "tillend"), null);
private final DimFilter in2 = new InDimFilter("dim2", ImmutableList.of("again"), null);
private final DimFilter in3 = new InDimFilter("dim1", Arrays.asList("null", null), null);
private final DimFilter bound1 = new BoundDimFilter("dim1", "from", "to", false, false, false, null);
private final DimFilter bound2 = new BoundDimFilter("dim1", null, "tillend", false, false, false, null);
private final DimFilter bound3 = new BoundDimFilter("dim1", "notincluded", null, true, false, false, null);
private final DimFilter bound4 = new BoundDimFilter("dim2", "again", "exclusive", true, true, false, null);
private final DimFilter bound1 = new BoundDimFilter("dim1", "from", "to", false, false, false, null,
StringComparators.LEXICOGRAPHIC
);
private final DimFilter bound2 = new BoundDimFilter("dim1", null, "tillend", false, false, false, null,
StringComparators.LEXICOGRAPHIC
);
private final DimFilter bound3 = new BoundDimFilter("dim1", "notincluded", null, true, false, false, null,
StringComparators.LEXICOGRAPHIC
);
private final DimFilter bound4 = new BoundDimFilter("dim2", "again", "exclusive", true, true, false, null,
StringComparators.LEXICOGRAPHIC
);
private final DimFilter other1 = new RegexDimFilter("someDim", "pattern", null);
private final DimFilter other2 = new JavaScriptDimFilter("someOtherDim", "function(x) { return x }", null,
JavaScriptConfig.getDefault());

View File

@ -2080,8 +2080,8 @@ public class GroupByQueryRunnerTest
new LongSumAggregatorFactory("idx", "index")
)
)
.addOrderByColumn("rows", "desc")
.addOrderByColumn("alias", "d")
.addOrderByColumn("rows", OrderByColumnSpec.Direction.DESCENDING)
.addOrderByColumn("alias", OrderByColumnSpec.Direction.DESCENDING)
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
final GroupByQuery query = builder.build();
@ -2120,8 +2120,8 @@ public class GroupByQueryRunnerTest
new DoubleSumAggregatorFactory("idx", "index")
)
)
.addOrderByColumn("idx", "desc")
.addOrderByColumn("alias", "d")
.addOrderByColumn("idx", OrderByColumnSpec.Direction.DESCENDING)
.addOrderByColumn("alias", OrderByColumnSpec.Direction.DESCENDING)
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
final GroupByQuery query = builder.build();
@ -2218,6 +2218,46 @@ public class GroupByQueryRunnerTest
);
}
@Test
public void testGroupByOrderLimitNumeric() throws Exception
{
GroupByQuery.Builder builder = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
.setInterval("2011-04-02/2011-04-04")
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
.setAggregatorSpecs(
Arrays.asList(
QueryRunnerTestHelper.rowsCount,
new LongSumAggregatorFactory("idx", "index")
)
)
.addOrderByColumn(new OrderByColumnSpec("rows", OrderByColumnSpec.Direction.DESCENDING, StringComparators.NUMERIC))
.addOrderByColumn(new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.ASCENDING, StringComparators.NUMERIC))
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
final GroupByQuery query = builder.build();
List<Row> expectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L)
);
Map<String, Object> context = Maps.newHashMap();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited"
);
}
@Test
public void testGroupByWithSameCaseOrdering()
{
@ -5476,7 +5516,8 @@ public class GroupByQueryRunnerTest
false,
false,
true,
extractionFn
extractionFn,
StringComparators.ALPHANUMERIC
));
superFilterList.add(new RegexDimFilter("quality", "super-mezzanine", extractionFn));
superFilterList.add(new SearchQueryDimFilter(
@ -5534,7 +5575,9 @@ public class GroupByQueryRunnerTest
List<DimFilter> superFilterList = new ArrayList<>();
superFilterList.add(new SelectorDimFilter("null_column", "EMPTY", extractionFn));
superFilterList.add(new InDimFilter("null_column", Arrays.asList("NOT-EMPTY", "FOOBAR", "EMPTY"), extractionFn));
superFilterList.add(new BoundDimFilter("null_column", "EMPTY", "EMPTY", false, false, true, extractionFn));
superFilterList.add(new BoundDimFilter("null_column", "EMPTY", "EMPTY", false, false, true, extractionFn,
StringComparators.ALPHANUMERIC
));
superFilterList.add(new RegexDimFilter("null_column", "EMPTY", extractionFn));
superFilterList.add(new SearchQueryDimFilter(
"null_column",

View File

@ -35,6 +35,7 @@ import io.druid.query.aggregation.post.ArithmeticPostAggregator;
import io.druid.query.aggregation.post.ConstantPostAggregator;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
import org.junit.Assert;
@ -83,7 +84,22 @@ public class DefaultLimitSpecTest
//non-defaults
json = "{\n"
+ " \"type\":\"default\",\n"
+ " \"columns\":[{\"dimension\":\"d\",\"direction\":\"ASCENDING\"}],\n"
+ " \"columns\":[{\"dimension\":\"d\",\"direction\":\"DESCENDING\", \"dimensionOrder\":\"numeric\"}],\n"
+ " \"limit\":10\n"
+ "}";
spec = mapper.readValue(
mapper.writeValueAsString(mapper.readValue(json, DefaultLimitSpec.class)),
DefaultLimitSpec.class
);
Assert.assertEquals(
new DefaultLimitSpec(ImmutableList.of(new OrderByColumnSpec("d", OrderByColumnSpec.Direction.DESCENDING,
StringComparators.NUMERIC)), 10),
spec
);
json = "{\n"
+ " \"type\":\"default\",\n"
+ " \"columns\":[{\"dimension\":\"d\",\"direction\":\"DES\", \"dimensionOrder\":\"numeric\"}],\n"
+ " \"limit\":10\n"
+ "}";
@ -93,9 +109,42 @@ public class DefaultLimitSpecTest
);
Assert.assertEquals(
new DefaultLimitSpec(ImmutableList.of(new OrderByColumnSpec("d", OrderByColumnSpec.Direction.ASCENDING)), 10),
new DefaultLimitSpec(ImmutableList.of(new OrderByColumnSpec("d", OrderByColumnSpec.Direction.DESCENDING,
StringComparators.NUMERIC)), 10),
spec
);
json = "{\n"
+ " \"type\":\"default\",\n"
+ " \"columns\":[{\"dimension\":\"d\"}],\n"
+ " \"limit\":10\n"
+ "}";
spec = mapper.readValue(
mapper.writeValueAsString(mapper.readValue(json, DefaultLimitSpec.class)),
DefaultLimitSpec.class
);
Assert.assertEquals(
new DefaultLimitSpec(ImmutableList.of(new OrderByColumnSpec("d", OrderByColumnSpec.Direction.ASCENDING,
StringComparators.LEXICOGRAPHIC)), 10),
spec
);
json = "{\n"
+ " \"type\":\"default\",\n"
+ " \"columns\":[\"d\"],\n"
+ " \"limit\":10\n"
+ "}";
spec = mapper.readValue(
mapper.writeValueAsString(mapper.readValue(json, DefaultLimitSpec.class)),
DefaultLimitSpec.class
);
Assert.assertEquals(
new DefaultLimitSpec(ImmutableList.of(new OrderByColumnSpec("d", OrderByColumnSpec.Direction.ASCENDING,
StringComparators.LEXICOGRAPHIC)), 10),
spec
);
}
@Test

View File

@ -20,6 +20,7 @@
package io.druid.query.ordering;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@ -31,11 +32,10 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.ordering.StringComparators.StringComparator;
import io.druid.query.ordering.StringComparator;
public class StringComparatorsTest
{
private void commonTest(StringComparator comparator)
{
// equality test
@ -119,16 +119,46 @@ public class StringComparatorsTest
Assert.assertTrue(StringComparators.STRLEN.compare("apple", "elppa") < 0);
}
@Test
public void testNumericComparator()
{
commonTest(StringComparators.NUMERIC);
Assert.assertTrue(StringComparators.NUMERIC.compare("-1230.452487532", "6893") < 0);
List<String> values = Arrays.asList("-1", "-1.10", "-1.2", "-100", "-2", "0", "1", "1.10", "1.2", "2", "100");
Collections.sort(values, StringComparators.NUMERIC);
Assert.assertEquals(
Arrays.asList("-100", "-2", "-1.2", "-1.10", "-1", "0", "1", "1.10", "1.2", "2", "100"),
values
);
Assert.assertTrue(StringComparators.NUMERIC.compare(null, null) == 0);
Assert.assertTrue(StringComparators.NUMERIC.compare(null, "1001") < 0);
Assert.assertTrue(StringComparators.NUMERIC.compare("1001", null) > 0);
Assert.assertTrue(StringComparators.NUMERIC.compare("-500000000.14124", "CAN'T TOUCH THIS") > 0);
Assert.assertTrue(StringComparators.NUMERIC.compare("CAN'T PARSE THIS", "-500000000.14124") < 0);
Assert.assertTrue(StringComparators.NUMERIC.compare("CAN'T PARSE THIS", "CAN'T TOUCH THIS") < 0);
}
@Test
public void testLexicographicComparatorSerdeTest() throws IOException
{
ObjectMapper jsonMapper = new DefaultObjectMapper();
String expectJsonSpec = "{\"type\":\"lexicographic\"}";
String jsonSpec = jsonMapper.writeValueAsString(StringComparators.LEXICOGRAPHIC);
Assert.assertEquals(expectJsonSpec, jsonSpec);
Assert.assertEquals(StringComparators.LEXICOGRAPHIC
, jsonMapper.readValue(expectJsonSpec, StringComparators.LexicographicComparator.class));
, jsonMapper.readValue(expectJsonSpec, StringComparator.class));
String makeFromJsonSpec = "\"lexicographic\"";
Assert.assertEquals(StringComparators.LEXICOGRAPHIC
, jsonMapper.readValue(makeFromJsonSpec, StringComparator.class));
}
@Test
@ -136,11 +166,15 @@ public class StringComparatorsTest
{
ObjectMapper jsonMapper = new DefaultObjectMapper();
String expectJsonSpec = "{\"type\":\"alphanumeric\"}";
String jsonSpec = jsonMapper.writeValueAsString(StringComparators.ALPHANUMERIC);
Assert.assertEquals(expectJsonSpec, jsonSpec);
Assert.assertEquals(StringComparators.ALPHANUMERIC
, jsonMapper.readValue(expectJsonSpec, StringComparators.AlphanumericComparator.class));
, jsonMapper.readValue(expectJsonSpec, StringComparator.class));
String makeFromJsonSpec = "\"alphanumeric\"";
Assert.assertEquals(StringComparators.ALPHANUMERIC
, jsonMapper.readValue(makeFromJsonSpec, StringComparator.class));
}
@Test
@ -152,6 +186,26 @@ public class StringComparatorsTest
String jsonSpec = jsonMapper.writeValueAsString(StringComparators.STRLEN);
Assert.assertEquals(expectJsonSpec, jsonSpec);
Assert.assertEquals(StringComparators.STRLEN
, jsonMapper.readValue(expectJsonSpec, StringComparators.StrlenComparator.class));
, jsonMapper.readValue(expectJsonSpec, StringComparator.class));
String makeFromJsonSpec = "\"strlen\"";
Assert.assertEquals(StringComparators.STRLEN
, jsonMapper.readValue(makeFromJsonSpec, StringComparator.class));
}
@Test
public void testNumericComparatorSerdeTest() throws IOException
{
ObjectMapper jsonMapper = new DefaultObjectMapper();
String expectJsonSpec = "{\"type\":\"numeric\"}";
String jsonSpec = jsonMapper.writeValueAsString(StringComparators.NUMERIC);
Assert.assertEquals(expectJsonSpec, jsonSpec);
Assert.assertEquals(StringComparators.NUMERIC
, jsonMapper.readValue(expectJsonSpec, StringComparator.class));
String makeFromJsonSpec = "\"numeric\"";
Assert.assertEquals(StringComparators.NUMERIC
, jsonMapper.readValue(makeFromJsonSpec, StringComparator.class));
}
}

View File

@ -1,45 +0,0 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.search;
import io.druid.query.search.search.AlphanumericSearchSortSpec;
import io.druid.query.search.search.SearchHit;
import io.druid.query.search.search.SearchSortSpec;
import org.junit.Assert;
import org.junit.Test;
/**
*/
public class AlphanumericSearchSortSpecTest
{
@Test
public void testComparator()
{
SearchSortSpec spec = new AlphanumericSearchSortSpec();
SearchHit hit1 = new SearchHit("test", "a100");
SearchHit hit2 = new SearchHit("test", "a9");
SearchHit hit3 = new SearchHit("test", "b0");
Assert.assertTrue(spec.getComparator().compare(hit1, hit2) > 0);
Assert.assertTrue(spec.getComparator().compare(hit3, hit1) > 0);
Assert.assertTrue(spec.getComparator().compare(hit3, hit2) > 0);
}
}

View File

@ -1,45 +0,0 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.search;
import io.druid.query.search.search.LexicographicSearchSortSpec;
import io.druid.query.search.search.SearchHit;
import io.druid.query.search.search.SearchSortSpec;
import org.junit.Assert;
import org.junit.Test;
/**
*/
public class LexicographicSearchSortSpecTest
{
@Test
public void testComparator()
{
SearchHit hit1 = new SearchHit("test", "apple");
SearchHit hit2 = new SearchHit("test", "banana");
SearchHit hit3 = new SearchHit("test", "banana");
SearchSortSpec spec = new LexicographicSearchSortSpec();
Assert.assertTrue(spec.getComparator().compare(hit2, hit3) == 0);
Assert.assertTrue(spec.getComparator().compare(hit2, hit1) > 0);
Assert.assertTrue(spec.getComparator().compare(hit1, hit3) < 0);
}
}

View File

@ -22,10 +22,9 @@ package io.druid.query.search;
import com.google.common.collect.ImmutableList;
import io.druid.granularity.QueryGranularities;
import io.druid.query.Result;
import io.druid.query.search.search.AlphanumericSearchSortSpec;
import io.druid.query.search.search.LexicographicSearchSortSpec;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.SearchSortSpec;
import io.druid.query.search.search.SearchHit;
import io.druid.query.search.search.StrlenSearchSortSpec;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
@ -98,7 +97,7 @@ public class SearchBinaryFnTest
)
);
Result<SearchResultValue> actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
assertSearchMergeResult(expected.getValue(), actual.getValue());
}
@ -146,7 +145,7 @@ public class SearchBinaryFnTest
)
);
Result<SearchResultValue> actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.DAY, Integer.MAX_VALUE).apply(r1, r2);
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), QueryGranularities.DAY, Integer.MAX_VALUE).apply(r1, r2);
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
assertSearchMergeResult(expected.getValue(), actual.getValue());
}
@ -170,7 +169,7 @@ public class SearchBinaryFnTest
Result<SearchResultValue> expected = r1;
Result<SearchResultValue> actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
assertSearchMergeResult(expected.getValue(), actual.getValue());
}
@ -218,7 +217,7 @@ public class SearchBinaryFnTest
)
);
Result<SearchResultValue> actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
assertSearchMergeResult(expected.getValue(), actual.getValue());
}
@ -226,7 +225,7 @@ public class SearchBinaryFnTest
@Test
public void testStrlenMerge()
{
StrlenSearchSortSpec searchSortSpec = new StrlenSearchSortSpec();
SearchSortSpec searchSortSpec = new SearchSortSpec(StringComparators.STRLEN);
Comparator<SearchHit> c = searchSortSpec.getComparator();
Result<SearchResultValue> r1 = new Result<SearchResultValue>(
@ -252,7 +251,7 @@ public class SearchBinaryFnTest
@Test
public void testStrlenMerge2()
{
StrlenSearchSortSpec searchSortSpec = new StrlenSearchSortSpec();
SearchSortSpec searchSortSpec = new SearchSortSpec(StringComparators.STRLEN);
Comparator<SearchHit> c = searchSortSpec.getComparator();
Result<SearchResultValue> r1 = new Result<SearchResultValue>(
@ -278,7 +277,7 @@ public class SearchBinaryFnTest
@Test
public void testAlphanumericMerge()
{
AlphanumericSearchSortSpec searchSortSpec = new AlphanumericSearchSortSpec();
SearchSortSpec searchSortSpec = new SearchSortSpec(StringComparators.ALPHANUMERIC);
Comparator<SearchHit> c = searchSortSpec.getComparator();
Result<SearchResultValue> r1 = new Result<SearchResultValue>(
@ -332,7 +331,7 @@ public class SearchBinaryFnTest
Result<SearchResultValue> expected = r1;
Result<SearchResultValue> actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
assertSearchMergeResult(expected.getValue(), actual.getValue());
}
@ -363,7 +362,7 @@ public class SearchBinaryFnTest
)
);
Result<SearchResultValue> expected = r1;
Result<SearchResultValue> actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, 1).apply(r1, r2);
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), QueryGranularities.ALL, 1).apply(r1, r2);
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
assertSearchMergeResult(expected.getValue(), actual.getValue());
}
@ -397,7 +396,7 @@ public class SearchBinaryFnTest
Result<SearchResultValue> expected = r1;
Result<SearchResultValue> actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
assertSearchMergeResult(expected.getValue(), actual.getValue());
}

View File

@ -19,7 +19,6 @@
package io.druid.query.search;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.metamx.common.guava.Sequence;
@ -36,13 +35,13 @@ import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.ExtractionDimFilter;
import io.druid.query.filter.RegexDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.FragmentSearchQuerySpec;
import io.druid.query.search.search.SearchSortSpec;
import io.druid.query.search.search.SearchHit;
import io.druid.query.search.search.SearchQuery;
import io.druid.query.search.search.SearchQueryConfig;
import io.druid.query.search.search.StrlenSearchSortSpec;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
@ -213,7 +212,7 @@ public class SearchQueryRunnerTest
QueryRunnerTestHelper.placementishDimension
)
)
.sortSpec(new StrlenSearchSortSpec())
.sortSpec(new SearchSortSpec(StringComparators.STRLEN))
.query("e")
.build();
@ -579,6 +578,31 @@ public class SearchQueryRunnerTest
);
}
@Test
public void testSearchWithNumericSort()
{
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.query("a")
.sortSpec(new SearchSortSpec(StringComparators.NUMERIC))
.build();
List<SearchHit> expectedHits = Lists.newLinkedList();
expectedHits.add(new SearchHit(QueryRunnerTestHelper.placementishDimension, "a", 93));
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "automotive", 93));
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "entertainment", 93));
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "health", 93));
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "mezzanine", 279));
expectedHits.add(new SearchHit(QueryRunnerTestHelper.marketDimension, "total_market", 186));
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "travel", 93));
expectedHits.add(new SearchHit(QueryRunnerTestHelper.partialNullDimension, "value", 186));
checkSearchQuery(searchQuery, expectedHits);
}
private void checkSearchQuery(Query searchQuery, List<SearchHit> expectedResults)
{
checkSearchQuery(searchQuery, runner, expectedResults);

View File

@ -0,0 +1,114 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.search;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.SearchSortSpec;
import io.druid.query.search.search.SearchHit;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
/**
*/
public class SearchSortSpecTest
{
@Test
public void testLexicographicComparator()
{
SearchHit hit1 = new SearchHit("test", "apple");
SearchHit hit2 = new SearchHit("test", "banana");
SearchHit hit3 = new SearchHit("test", "banana");
SearchSortSpec spec = new SearchSortSpec(StringComparators.LEXICOGRAPHIC);
Assert.assertTrue(spec.getComparator().compare(hit2, hit3) == 0);
Assert.assertTrue(spec.getComparator().compare(hit2, hit1) > 0);
Assert.assertTrue(spec.getComparator().compare(hit1, hit3) < 0);
}
@Test
public void testAlphanumericComparator()
{
SearchSortSpec spec = new SearchSortSpec(StringComparators.ALPHANUMERIC);
SearchHit hit1 = new SearchHit("test", "a100");
SearchHit hit2 = new SearchHit("test", "a9");
SearchHit hit3 = new SearchHit("test", "b0");
Assert.assertTrue(spec.getComparator().compare(hit1, hit2) > 0);
Assert.assertTrue(spec.getComparator().compare(hit3, hit1) > 0);
Assert.assertTrue(spec.getComparator().compare(hit3, hit2) > 0);
}
@Test
public void testNumericComparator()
{
SearchSortSpec spec = new SearchSortSpec(StringComparators.NUMERIC);
SearchHit hit1 = new SearchHit("test", "1001001.12412");
SearchHit hit2 = new SearchHit("test", "-1421");
SearchHit hit3 = new SearchHit("test", "not-numeric-at-all");
SearchHit hit4 = new SearchHit("best", "1001001.12412");
Assert.assertTrue(spec.getComparator().compare(hit1, hit2) > 0);
Assert.assertTrue(spec.getComparator().compare(hit3, hit1) < 0);
Assert.assertTrue(spec.getComparator().compare(hit3, hit2) < 0);
Assert.assertTrue(spec.getComparator().compare(hit1, hit4) > 0);
}
@Test
public void testStrlenComparator()
{
SearchSortSpec spec = new SearchSortSpec(StringComparators.LEXICOGRAPHIC);
SearchHit hit1 = new SearchHit("test", "apple");
SearchHit hit2 = new SearchHit("test", "banana");
SearchHit hit3 = new SearchHit("test", "orange");
Assert.assertTrue(spec.getComparator().compare(hit1, hit2) < 0);
Assert.assertTrue(spec.getComparator().compare(hit3, hit1) > 0);
Assert.assertTrue(spec.getComparator().compare(hit3, hit2) > 0);
Assert.assertTrue(spec.getComparator().compare(hit1, hit1) == 0);
}
@Test
public void testSerde() throws IOException
{
ObjectMapper jsonMapper = new DefaultObjectMapper();
SearchSortSpec spec = new SearchSortSpec(StringComparators.ALPHANUMERIC);
String expectJsonSpec = "{\"type\":{\"type\":\"alphanumeric\"}}";
String jsonSpec = jsonMapper.writeValueAsString(spec);
Assert.assertEquals(expectJsonSpec, jsonSpec);
Assert.assertEquals(spec, jsonMapper.readValue(jsonSpec, SearchSortSpec.class));
String expectJsonSpec2 = "{\"type\":\"alphanumeric\"}";
Assert.assertEquals(spec, jsonMapper.readValue(expectJsonSpec2, SearchSortSpec.class));
}
}

View File

@ -47,6 +47,7 @@ import io.druid.query.filter.NotDimFilter;
import io.druid.query.filter.RegexDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.ordering.StringComparators;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
@ -2238,7 +2239,8 @@ public class TimeseriesQueryRunnerTest
true,
null,
null,
null
null,
StringComparators.LEXICOGRAPHIC
),
new BoundDimFilter(
QueryRunnerTestHelper.marketDimension,
@ -2247,7 +2249,8 @@ public class TimeseriesQueryRunnerTest
null,
true,
null,
null
null,
StringComparators.LEXICOGRAPHIC
),
(DimFilter) new BoundDimFilter(
QueryRunnerTestHelper.marketDimension,
@ -2256,7 +2259,8 @@ public class TimeseriesQueryRunnerTest
null,
null,
null,
null
null,
StringComparators.LEXICOGRAPHIC
)
)
)

View File

@ -0,0 +1,116 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.topn;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.ordering.StringComparators;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
public class DimensionTopNMetricSpecTest
{
@Test
public void testSerdeAlphaNumericDimensionTopNMetricSpec() throws IOException{
DimensionTopNMetricSpec expectedMetricSpec = new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC);
DimensionTopNMetricSpec expectedMetricSpec1 = new DimensionTopNMetricSpec("test", StringComparators.ALPHANUMERIC);
String jsonSpec = "{\n"
+ " \"type\": \"dimension\","
+ " \"ordering\": \"alphanumeric\"\n"
+ "}";
String jsonSpec1 = "{\n"
+ " \"type\": \"dimension\","
+ " \"ordering\": \"alphanumeric\",\n"
+ " \"previousStop\": \"test\"\n"
+ "}";
ObjectMapper jsonMapper = new DefaultObjectMapper();
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
Assert.assertEquals(expectedMetricSpec1, actualMetricSpec1);
}
@Test
public void testSerdeLexicographicDimensionTopNMetricSpec() throws IOException{
DimensionTopNMetricSpec expectedMetricSpec = new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC);
DimensionTopNMetricSpec expectedMetricSpec1 = new DimensionTopNMetricSpec("test", StringComparators.LEXICOGRAPHIC);
String jsonSpec = "{\n"
+ " \"type\": \"dimension\","
+ " \"ordering\": \"lexicographic\"\n"
+ "}";
String jsonSpec1 = "{\n"
+ " \"type\": \"dimension\","
+ " \"ordering\": \"lexicographic\",\n"
+ " \"previousStop\": \"test\"\n"
+ "}";
ObjectMapper jsonMapper = new DefaultObjectMapper();
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
Assert.assertEquals(expectedMetricSpec1, actualMetricSpec1);
}
@Test
public void testSerdeStrlenDimensionTopNMetricSpec() throws IOException{
DimensionTopNMetricSpec expectedMetricSpec = new DimensionTopNMetricSpec(null, StringComparators.STRLEN);
DimensionTopNMetricSpec expectedMetricSpec1 = new DimensionTopNMetricSpec("test", StringComparators.STRLEN);
String jsonSpec = "{\n"
+ " \"type\": \"dimension\","
+ " \"ordering\": \"strlen\"\n"
+ "}";
String jsonSpec1 = "{\n"
+ " \"type\": \"dimension\","
+ " \"ordering\": \"strlen\",\n"
+ " \"previousStop\": \"test\"\n"
+ "}";
ObjectMapper jsonMapper = new DefaultObjectMapper();
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
Assert.assertEquals(expectedMetricSpec1, actualMetricSpec1);
}
@Test
public void testSerdeNumericDimensionTopNMetricSpec() throws IOException{
DimensionTopNMetricSpec expectedMetricSpec = new DimensionTopNMetricSpec(null, StringComparators.NUMERIC);
DimensionTopNMetricSpec expectedMetricSpec1 = new DimensionTopNMetricSpec("test", StringComparators.NUMERIC);
String jsonSpec = "{\n"
+ " \"type\": \"dimension\","
+ " \"ordering\": \"numeric\"\n"
+ "}";
String jsonSpec1 = "{\n"
+ " \"type\": \"dimension\","
+ " \"ordering\": \"numeric\",\n"
+ " \"previousStop\": \"test\"\n"
+ "}";
ObjectMapper jsonMapper = new DefaultObjectMapper();
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
Assert.assertEquals(expectedMetricSpec1, actualMetricSpec1);
}
}

View File

@ -32,6 +32,7 @@ import io.druid.query.aggregation.post.ArithmeticPostAggregator;
import io.druid.query.aggregation.post.ConstantPostAggregator;
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.ordering.StringComparators;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
@ -509,7 +510,7 @@ public class TopNBinaryFnTest
TopNResultMerger.identity,
QueryGranularities.ALL,
new DefaultDimensionSpec("INVALID_DIM_NAME", null),
new LexicographicTopNMetricSpec(null),
new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC),
2,
aggregatorFactories,
postAggregators

View File

@ -62,6 +62,7 @@ import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.ExtractionDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.query.timeseries.TimeseriesQuery;
import io.druid.segment.TestHelper;
@ -1299,7 +1300,7 @@ public class TopNQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(new LexicographicTopNMetricSpec(""))
.metric(new DimensionTopNMetricSpec("", StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1346,7 +1347,7 @@ public class TopNQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(new LexicographicTopNMetricSpec(""))
.metric(new DimensionTopNMetricSpec("", StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.build();
@ -1379,7 +1380,7 @@ public class TopNQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(new LexicographicTopNMetricSpec("spot"))
.metric(new DimensionTopNMetricSpec("spot", StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1419,7 +1420,7 @@ public class TopNQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(new LexicographicTopNMetricSpec("t"))
.metric(new DimensionTopNMetricSpec("t", StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1459,7 +1460,7 @@ public class TopNQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("upfront")))
.metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec("upfront", StringComparators.LEXICOGRAPHIC)))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1499,7 +1500,7 @@ public class TopNQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("u")))
.metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec("u", StringComparators.LEXICOGRAPHIC)))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1986,7 +1987,7 @@ public class TopNQueryRunnerTest
null
)
)
.metric(new LexicographicTopNMetricSpec(null))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -2053,7 +2054,7 @@ public class TopNQueryRunnerTest
null
)
)
.metric(new LexicographicTopNMetricSpec(null))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -2121,7 +2122,7 @@ public class TopNQueryRunnerTest
null
)
)
.metric(new LexicographicTopNMetricSpec(null))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -2175,7 +2176,7 @@ public class TopNQueryRunnerTest
null
)
)
.metric(new LexicographicTopNMetricSpec(null))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -2229,7 +2230,7 @@ public class TopNQueryRunnerTest
null
)
)
.metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec(null)))
.metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -2283,7 +2284,7 @@ public class TopNQueryRunnerTest
null
)
)
.metric(new LexicographicTopNMetricSpec("s"))
.metric(new DimensionTopNMetricSpec("s", StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -2353,7 +2354,7 @@ public class TopNQueryRunnerTest
}, null
)
)
.metric(new LexicographicTopNMetricSpec("s"))
.metric(new DimensionTopNMetricSpec("s", StringComparators.LEXICOGRAPHIC))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -2401,7 +2402,7 @@ public class TopNQueryRunnerTest
null
)
)
.metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("u")))
.metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec("u", StringComparators.LEXICOGRAPHIC)))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -2448,7 +2449,7 @@ public class TopNQueryRunnerTest
null
)
)
.metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("p")))
.metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec("p", StringComparators.LEXICOGRAPHIC)))
.threshold(4)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -3225,7 +3226,7 @@ public class TopNQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryGranularities.ALL)
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(new AlphaNumericTopNMetricSpec(null))
.metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC))
.threshold(2)
.intervals(QueryRunnerTestHelper.secondOnly)
.aggregators(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
@ -3250,6 +3251,39 @@ public class TopNQueryRunnerTest
TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap<String, Object>()));
}
@Test
public void testNumericDimensionTopNWithNullPreviousStop()
{
TopNQuery query = new TopNQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryGranularities.ALL)
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
.threshold(2)
.intervals(QueryRunnerTestHelper.secondOnly)
.aggregators(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
.build();
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
new Result<>(
new DateTime("2011-04-02T00:00:00.000Z"),
new TopNResultValue(
Arrays.asList(
ImmutableMap.<String, Object>of(
"market", "spot",
"rows", 9L
),
ImmutableMap.<String, Object>of(
"market", "total_market",
"rows", 2L
)
)
)
)
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap<String, Object>()));
}
@Test
public void testTopNWithExtractionFilter()
{

View File

@ -33,6 +33,7 @@ import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.dimension.LegacyDimensionSpec;
import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.ordering.StringComparators;
import org.junit.Assert;
import org.junit.Test;
@ -123,7 +124,7 @@ public class TopNQueryTest
.dataSource(dataSource)
.granularity(allGran)
.dimension(new LegacyDimensionSpec(marketDimension))
.metric(new AlphaNumericTopNMetricSpec(null))
.metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC))
.threshold(2)
.intervals(fullOnInterval.getIntervals())
.aggregators(Lists.<AggregatorFactory>newArrayList(rowsCount))
@ -134,7 +135,8 @@ public class TopNQueryTest
+ " \"dimension\": \"market\",\n"
+ " \"threshold\": 2,\n"
+ " \"metric\": {\n"
+ " \"type\": \"alphaNumeric\"\n"
+ " \"type\": \"dimension\",\n"
+ " \"ordering\": \"alphanumeric\"\n"
+ " },\n"
+ " \"granularity\": \"all\",\n"
+ " \"aggregations\": [\n"

View File

@ -34,6 +34,7 @@ import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.JavaScriptExtractionFn;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.IndexBuilder;
import io.druid.segment.StorageAdapter;
import org.joda.time.DateTime;
@ -65,7 +66,9 @@ public class BoundFilterTest extends BaseFilterTest
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "2", "dim1", "2", "dim2", ImmutableList.of(""))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "3", "dim1", "1", "dim2", ImmutableList.of("a"))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "4", "dim1", "def", "dim2", ImmutableList.of("c"))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "5", "dim1", "abc"))
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "5", "dim1", "abc")),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "6", "dim1", "-1000", "dim2", ImmutableList.of("a"))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "7", "dim1", "-10.012", "dim2", ImmutableList.of("d")))
);
public BoundFilterTest(
@ -88,14 +91,14 @@ public class BoundFilterTest extends BaseFilterTest
public void testLexicographicMatchEverything()
{
final List<BoundDimFilter> filters = ImmutableList.of(
new BoundDimFilter("dim0", "", "z", false, false, false, null),
new BoundDimFilter("dim1", "", "z", false, false, false, null),
new BoundDimFilter("dim2", "", "z", false, false, false, null),
new BoundDimFilter("dim3", "", "z", false, false, false, null)
new BoundDimFilter("dim0", "", "z", false, false, false, null, StringComparators.LEXICOGRAPHIC),
new BoundDimFilter("dim1", "", "z", false, false, false, null, StringComparators.LEXICOGRAPHIC),
new BoundDimFilter("dim2", "", "z", false, false, false, null, StringComparators.LEXICOGRAPHIC),
new BoundDimFilter("dim3", "", "z", false, false, false, null, StringComparators.LEXICOGRAPHIC)
);
for (BoundDimFilter filter : filters) {
assertFilterMatches(filter, ImmutableList.of("0", "1", "2", "3", "4", "5"));
assertFilterMatches(filter, ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7"));
}
}
@ -103,15 +106,15 @@ public class BoundFilterTest extends BaseFilterTest
public void testLexicographicMatchNull()
{
assertFilterMatches(
new BoundDimFilter("dim0", "", "", false, false, false, null),
new BoundDimFilter("dim0", "", "", false, false, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "", "", false, false, false, null),
new BoundDimFilter("dim1", "", "", false, false, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("0")
);
assertFilterMatches(
new BoundDimFilter("dim2", "", "", false, false, false, null),
new BoundDimFilter("dim2", "", "", false, false, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("1", "2", "5")
);
}
@ -120,44 +123,45 @@ public class BoundFilterTest extends BaseFilterTest
public void testLexicographicMatchMissingColumn()
{
assertFilterMatches(
new BoundDimFilter("dim3", "", "", false, false, false, null),
ImmutableList.of("0", "1", "2", "3", "4", "5")
new BoundDimFilter("dim3", "", "", false, false, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
assertFilterMatches(
new BoundDimFilter("dim3", "", "", true, false, false, null),
new BoundDimFilter("dim3", "", "", true, false, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim3", "", "", false, true, false, null),
new BoundDimFilter("dim3", "", "", false, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim3", "", null, false, true, false, null),
ImmutableList.of("0", "1", "2", "3", "4", "5")
new BoundDimFilter("dim3", "", null, false, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
assertFilterMatches(
new BoundDimFilter("dim3", null, "", false, false, false, null),
ImmutableList.of("0", "1", "2", "3", "4", "5")
new BoundDimFilter("dim3", null, "", false, false, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
assertFilterMatches(
new BoundDimFilter("dim3", null, "", false, true, false, null),
new BoundDimFilter("dim3", null, "", false, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of()
);
}
@Test
public void testLexicographicMatchTooStrict()
{
assertFilterMatches(
new BoundDimFilter("dim1", "abc", "abc", true, false, false, null),
new BoundDimFilter("dim1", "abc", "abc", true, false, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "abc", "abc", true, true, false, null),
new BoundDimFilter("dim1", "abc", "abc", true, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "abc", "abc", false, true, false, null),
new BoundDimFilter("dim1", "abc", "abc", false, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of()
);
}
@ -166,7 +170,7 @@ public class BoundFilterTest extends BaseFilterTest
public void testLexicographicMatchExactlySingleValue()
{
assertFilterMatches(
new BoundDimFilter("dim1", "abc", "abc", false, false, false, null),
new BoundDimFilter("dim1", "abc", "abc", false, false, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("5")
);
}
@ -175,7 +179,7 @@ public class BoundFilterTest extends BaseFilterTest
public void testLexicographicMatchSurroundingSingleValue()
{
assertFilterMatches(
new BoundDimFilter("dim1", "ab", "abd", true, true, false, null),
new BoundDimFilter("dim1", "ab", "abd", true, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("5")
);
}
@ -184,7 +188,7 @@ public class BoundFilterTest extends BaseFilterTest
public void testLexicographicMatchNoUpperLimit()
{
assertFilterMatches(
new BoundDimFilter("dim1", "ab", null, true, true, false, null),
new BoundDimFilter("dim1", "ab", null, true, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("4", "5")
);
}
@ -193,8 +197,8 @@ public class BoundFilterTest extends BaseFilterTest
public void testLexicographicMatchNoLowerLimit()
{
assertFilterMatches(
new BoundDimFilter("dim1", null, "abd", true, true, false, null),
ImmutableList.of("0", "1", "2", "3", "5")
new BoundDimFilter("dim1", null, "abd", true, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("0", "1", "2", "3", "5", "6", "7")
);
}
@ -202,33 +206,37 @@ public class BoundFilterTest extends BaseFilterTest
public void testLexicographicMatchNumbers()
{
assertFilterMatches(
new BoundDimFilter("dim1", "1", "3", false, false, false, null),
new BoundDimFilter("dim1", "1", "3", false, false, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("1", "2", "3")
);
assertFilterMatches(
new BoundDimFilter("dim1", "1", "3", true, true, false, null),
new BoundDimFilter("dim1", "1", "3", true, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("1", "2")
);
assertFilterMatches(
new BoundDimFilter("dim1", "-1", "3", true, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("1", "2", "3", "6", "7")
);
}
@Test
public void testAlphaNumericMatchNull()
{
assertFilterMatches(
new BoundDimFilter("dim0", "", "", false, false, true, null),
new BoundDimFilter("dim0", "", "", false, false, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "", "", false, false, true, null),
new BoundDimFilter("dim1", "", "", false, false, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("0")
);
assertFilterMatches(
new BoundDimFilter("dim2", "", "", false, false, true, null),
new BoundDimFilter("dim2", "", "", false, false, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("1", "2", "5")
);
assertFilterMatches(
new BoundDimFilter("dim3", "", "", false, false, true, null),
ImmutableList.of("0", "1", "2", "3", "4", "5")
new BoundDimFilter("dim3", "", "", false, false, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
}
@ -236,15 +244,15 @@ public class BoundFilterTest extends BaseFilterTest
public void testAlphaNumericMatchTooStrict()
{
assertFilterMatches(
new BoundDimFilter("dim1", "2", "2", true, false, true, null),
new BoundDimFilter("dim1", "2", "2", true, false, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "2", "2", true, true, true, null),
new BoundDimFilter("dim1", "2", "2", true, true, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "2", "2", false, true, true, null),
new BoundDimFilter("dim1", "2", "2", false, true, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of()
);
}
@ -253,7 +261,7 @@ public class BoundFilterTest extends BaseFilterTest
public void testAlphaNumericMatchExactlySingleValue()
{
assertFilterMatches(
new BoundDimFilter("dim1", "2", "2", false, false, true, null),
new BoundDimFilter("dim1", "2", "2", false, false, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("2")
);
}
@ -262,7 +270,7 @@ public class BoundFilterTest extends BaseFilterTest
public void testAlphaNumericMatchSurroundingSingleValue()
{
assertFilterMatches(
new BoundDimFilter("dim1", "1", "3", true, true, true, null),
new BoundDimFilter("dim1", "1", "3", true, true, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("2")
);
}
@ -271,8 +279,13 @@ public class BoundFilterTest extends BaseFilterTest
public void testAlphaNumericMatchNoUpperLimit()
{
assertFilterMatches(
new BoundDimFilter("dim1", "1", null, true, true, true, null),
ImmutableList.of("1", "2", "4", "5")
new BoundDimFilter("dim1", "1", null, true, true, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("1", "2", "4", "5", "6", "7")
);
assertFilterMatches(
new BoundDimFilter("dim1", "-1", null, true, true, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("4", "5", "6", "7")
);
}
@ -280,9 +293,121 @@ public class BoundFilterTest extends BaseFilterTest
public void testAlphaNumericMatchNoLowerLimit()
{
assertFilterMatches(
new BoundDimFilter("dim1", null, "2", true, true, true, null),
new BoundDimFilter("dim1", null, "2", true, true, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("0", "3")
);
assertFilterMatches(
new BoundDimFilter("dim1", null, "ZZZZZ", true, true, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
}
@Test
public void testAlphaNumericMatchWithNegatives()
{
assertFilterMatches(
new BoundDimFilter("dim1", "-2000", "3", true, true, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "3", "-2000", true, true, true, null, StringComparators.ALPHANUMERIC),
ImmutableList.of("1", "6", "7")
);
}
@Test
public void testNumericMatchNull()
{
assertFilterMatches(
new BoundDimFilter("dim0", "", "", false, false, false, null, StringComparators.NUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "", "", false, false, false, null, StringComparators.NUMERIC),
ImmutableList.of("0")
);
assertFilterMatches(
new BoundDimFilter("dim2", "", "", false, false, false, null, StringComparators.NUMERIC),
ImmutableList.of("1", "2", "5")
);
assertFilterMatches(
new BoundDimFilter("dim3", "", "", false, false, false, null, StringComparators.NUMERIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
}
@Test
public void testNumericMatchTooStrict()
{
assertFilterMatches(
new BoundDimFilter("dim1", "2", "2", true, false, false, null, StringComparators.NUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "2", "2", true, true, false, null, StringComparators.NUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter("dim1", "2", "2", false, true, false, null, StringComparators.NUMERIC),
ImmutableList.<String>of()
);
}
@Test
public void testNumericMatchExactlySingleValue()
{
assertFilterMatches(
new BoundDimFilter("dim1", "2", "2", false, false, false, null, StringComparators.NUMERIC),
ImmutableList.of("2")
);
assertFilterMatches(
new BoundDimFilter("dim1", "-10.012", "-10.012", false, false, false, null, StringComparators.NUMERIC),
ImmutableList.of("7")
);
}
@Test
public void testNumericMatchSurroundingSingleValue()
{
assertFilterMatches(
new BoundDimFilter("dim1", "1", "3", true, true, false, null, StringComparators.NUMERIC),
ImmutableList.of("2")
);
assertFilterMatches(
new BoundDimFilter("dim1", "-11", "-10", false, false, false, null, StringComparators.NUMERIC),
ImmutableList.of("7")
);
}
@Test
public void testNumericMatchNoUpperLimit()
{
assertFilterMatches(
new BoundDimFilter("dim1", "1", null, true, true, false, null, StringComparators.NUMERIC),
ImmutableList.of("1", "2")
);
}
@Test
public void testNumericMatchNoLowerLimit()
{
assertFilterMatches(
new BoundDimFilter("dim1", null, "2", true, true, false, null, StringComparators.NUMERIC),
ImmutableList.of("0", "3", "4", "5", "6", "7")
);
}
@Test
public void testNumericMatchWithNegatives()
{
assertFilterMatches(
new BoundDimFilter("dim1", "-2000", "3", true, true, false, null, StringComparators.NUMERIC),
ImmutableList.of("2", "3", "6", "7")
);
}
@Test
@ -295,38 +420,48 @@ public class BoundFilterTest extends BaseFilterTest
ExtractionFn makeNullFn = new JavaScriptExtractionFn(nullJsFn, false, JavaScriptConfig.getDefault());
assertFilterMatches(
new BoundDimFilter("dim0", "", "", false, false, false, makeNullFn),
ImmutableList.of("0", "1", "2", "3", "4", "5")
new BoundDimFilter("dim0", "", "", false, false, false, makeNullFn, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
assertFilterMatches(
new BoundDimFilter("dim1", "super-ab", "super-abd", true, true, false, superFn),
new BoundDimFilter("dim1", "super-ab", "super-abd", true, true, false, superFn, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("5")
);
assertFilterMatches(
new BoundDimFilter("dim1", "super-0", "super-10", false, false, true, superFn),
new BoundDimFilter("dim1", "super-0", "super-10", false, false, true, superFn, StringComparators.ALPHANUMERIC),
ImmutableList.of("1", "2", "3")
);
assertFilterMatches(
new BoundDimFilter("dim2", "super-", "super-zzzzzz", false, false, false, superFn),
ImmutableList.of("0", "1", "2", "3", "4", "5")
new BoundDimFilter("dim2", "super-", "super-zzzzzz", false, false, false, superFn, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
assertFilterMatches(
new BoundDimFilter("dim2", "super-null", "super-null", false, false, false, superFn),
new BoundDimFilter("dim2", "super-null", "super-null", false, false, false, superFn, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("1", "2", "5")
);
assertFilterMatches(
new BoundDimFilter("dim3", "super-null", "super-null", false, false, false, superFn),
ImmutableList.of("0", "1", "2", "3", "4", "5")
new BoundDimFilter("dim3", "super-null", "super-null", false, false, false, superFn, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
assertFilterMatches(
new BoundDimFilter("dim4", "super-null", "super-null", false, false, false, superFn),
ImmutableList.of("0", "1", "2", "3", "4", "5")
new BoundDimFilter("dim4", "super-null", "super-null", false, false, false, superFn, StringComparators.LEXICOGRAPHIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
assertFilterMatches(
new BoundDimFilter("dim2", "super-null", "super-null", false, false, false, superFn, StringComparators.NUMERIC),
ImmutableList.of("1", "2", "5")
);
assertFilterMatches(
new BoundDimFilter("dim4", "super-null", "super-null", false, false, false, superFn, StringComparators.NUMERIC),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
}

View File

@ -44,6 +44,7 @@ import io.druid.query.filter.SearchQueryDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.lookup.LookupExtractor;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import io.druid.segment.IndexBuilder;
import io.druid.segment.StorageAdapter;
@ -131,12 +132,12 @@ public class LongFilteringTest extends BaseFilterTest
);
assertFilterMatches(
new BoundDimFilter(COUNT_COLUMN, "2", "5", false, false, true, null),
new BoundDimFilter(COUNT_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("2", "3", "4", "5")
);
assertFilterMatches(
new BoundDimFilter(COUNT_COLUMN, "1", "4", true, true, true, null),
new BoundDimFilter(COUNT_COLUMN, "1", "4", true, true, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("2", "3")
);
@ -195,11 +196,11 @@ public class LongFilteringTest extends BaseFilterTest
);
assertFilterMatches(
new BoundDimFilter(COUNT_COLUMN, "Fridax", "Fridaz", false, false, true, exfn),
new BoundDimFilter(COUNT_COLUMN, "Fridax", "Fridaz", false, false, null, exfn, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of("5")
);
assertFilterMatches(
new BoundDimFilter(COUNT_COLUMN, "Friday", "Friday", true, true, true, exfn),
new BoundDimFilter(COUNT_COLUMN, "Friday", "Friday", true, true, null, exfn, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of()
);
@ -237,7 +238,7 @@ public class LongFilteringTest extends BaseFilterTest
}
@Test
public void testSelectorAndInFilterMultithreaded()
public void testMultithreaded()
{
assertFilterMatchesMultithreaded(
new SelectorDimFilter(COUNT_COLUMN, "3", null),
@ -258,6 +259,11 @@ public class LongFilteringTest extends BaseFilterTest
new InDimFilter(COUNT_COLUMN, infilterValues, null),
ImmutableList.<String>of("2", "4", "6")
);
assertFilterMatches(
new BoundDimFilter(COUNT_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("2", "3", "4", "5")
);
}
private void assertFilterMatches(

View File

@ -42,6 +42,7 @@ import io.druid.query.filter.SearchQueryDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.lookup.LookupExtractor;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import io.druid.segment.IndexBuilder;
import io.druid.segment.StorageAdapter;
@ -115,11 +116,11 @@ public class TimeFilteringTest extends BaseFilterTest
);
assertFilterMatches(
new BoundDimFilter(Column.TIME_COLUMN_NAME, "0", "4", false, false, true, null),
new BoundDimFilter(Column.TIME_COLUMN_NAME, "0", "4", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("0", "1", "2", "3", "4")
);
assertFilterMatches(
new BoundDimFilter(Column.TIME_COLUMN_NAME, "0", "4", true, true, true, null),
new BoundDimFilter(Column.TIME_COLUMN_NAME, "0", "4", true, true, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("1", "2", "3")
);
@ -178,11 +179,11 @@ public class TimeFilteringTest extends BaseFilterTest
);
assertFilterMatches(
new BoundDimFilter(Column.TIME_COLUMN_NAME, "Fridax", "Fridaz", false, false, true, exfn),
new BoundDimFilter(Column.TIME_COLUMN_NAME, "Fridax", "Fridaz", false, false, null, exfn, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of("4")
);
assertFilterMatches(
new BoundDimFilter(Column.TIME_COLUMN_NAME, "Friday", "Friday", true, true, true, exfn),
new BoundDimFilter(Column.TIME_COLUMN_NAME, "Friday", "Friday", true, true, null, exfn, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of()
);

View File

@ -97,6 +97,7 @@ import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryEngine;
import io.druid.query.groupby.GroupByQueryQueryToolChest;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.SearchQueryQueryToolChest;
import io.druid.query.search.SearchResultValue;
import io.druid.query.search.search.SearchHit;
@ -1403,14 +1404,20 @@ public class CachingClusteredClientTest
Druids.newOrDimFilterBuilder().fields(
Arrays.asList(
new SelectorDimFilter("dim0", "1", null),
new BoundDimFilter("dim0", "222", "333", false, false, false, null)
new BoundDimFilter("dim0", "222", "333", false, false, false, null,
StringComparators.LEXICOGRAPHIC
)
)
).build(),
Druids.newAndDimFilterBuilder().fields(
Arrays.asList(
new InDimFilter("dim1", Arrays.asList("0", "1", "2", "3", "4"), null),
new BoundDimFilter("dim1", "0", "3", false, true, false, null),
new BoundDimFilter("dim1", "1", "9999", true, false, false, null)
new BoundDimFilter("dim1", "0", "3", false, true, false, null,
StringComparators.LEXICOGRAPHIC
),
new BoundDimFilter("dim1", "1", "9999", true, false, false, null,
StringComparators.LEXICOGRAPHIC
)
)
).build()
)
@ -1475,14 +1482,20 @@ public class CachingClusteredClientTest
Druids.newOrDimFilterBuilder().fields(
Arrays.asList(
new SelectorDimFilter("dim1", "a", null),
new BoundDimFilter("dim1", "from", "to", false, false, false, null)
new BoundDimFilter("dim1", "from", "to", false, false, false, null,
StringComparators.LEXICOGRAPHIC
)
)
).build(),
Druids.newAndDimFilterBuilder().fields(
Arrays.asList(
new InDimFilter("dim2", Arrays.asList("a", "c", "e", "g"), null),
new BoundDimFilter("dim2", "aaa", "hi", false, false, false, null),
new BoundDimFilter("dim2", "e", "zzz", true, true, false, null)
new BoundDimFilter("dim2", "aaa", "hi", false, false, false, null,
StringComparators.LEXICOGRAPHIC
),
new BoundDimFilter("dim2", "e", "zzz", true, true, false, null,
StringComparators.LEXICOGRAPHIC
)
)
).build()
)