mirror of https://github.com/apache/druid.git
Allow filters to use extraction functions
This commit is contained in:
parent
18b9ea62cf
commit
0e481d6f93
|
@ -74,7 +74,8 @@ public class BoundFilterBenchmark
|
||||||
String.valueOf(START_INT),
|
String.valueOf(START_INT),
|
||||||
true,
|
true,
|
||||||
false,
|
false,
|
||||||
false
|
false,
|
||||||
|
null
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -85,7 +86,8 @@ public class BoundFilterBenchmark
|
||||||
String.valueOf(END_INT),
|
String.valueOf(END_INT),
|
||||||
false,
|
false,
|
||||||
false,
|
false,
|
||||||
false
|
false,
|
||||||
|
null
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -96,7 +98,8 @@ public class BoundFilterBenchmark
|
||||||
String.valueOf(END_INT),
|
String.valueOf(END_INT),
|
||||||
false,
|
false,
|
||||||
false,
|
false,
|
||||||
false
|
false,
|
||||||
|
null
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -107,7 +110,8 @@ public class BoundFilterBenchmark
|
||||||
String.valueOf(START_INT),
|
String.valueOf(START_INT),
|
||||||
true,
|
true,
|
||||||
false,
|
false,
|
||||||
true
|
true,
|
||||||
|
null
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -118,7 +122,8 @@ public class BoundFilterBenchmark
|
||||||
String.valueOf(END_INT),
|
String.valueOf(END_INT),
|
||||||
false,
|
false,
|
||||||
false,
|
false,
|
||||||
true
|
true,
|
||||||
|
null
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -129,7 +134,8 @@ public class BoundFilterBenchmark
|
||||||
String.valueOf(END_INT),
|
String.valueOf(END_INT),
|
||||||
false,
|
false,
|
||||||
false,
|
false,
|
||||||
true
|
true,
|
||||||
|
null
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -259,7 +259,7 @@ For instance the following filter
|
||||||
```json
|
```json
|
||||||
{
|
{
|
||||||
"filter": {
|
"filter": {
|
||||||
"type": "extraction",
|
"type": "selector",
|
||||||
"dimension": "product",
|
"dimension": "product",
|
||||||
"value": "bar_1",
|
"value": "bar_1",
|
||||||
"extractionFn": {
|
"extractionFn": {
|
||||||
|
|
|
@ -16,6 +16,8 @@ The grammar for a SELECTOR filter is as follows:
|
||||||
|
|
||||||
This is the equivalent of `WHERE <dimension_string> = '<dimension_value_string>'`.
|
This is the equivalent of `WHERE <dimension_string> = '<dimension_value_string>'`.
|
||||||
|
|
||||||
|
The selector filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
||||||
|
|
||||||
### Regular expression filter
|
### Regular expression filter
|
||||||
|
|
||||||
The regular expression filter is similar to the selector filter, but using regular expressions. It matches the specified dimension with the given pattern. The pattern can be any standard [Java regular expression](http://docs.oracle.com/javase/6/docs/api/java/util/regex/Pattern.html).
|
The regular expression filter is similar to the selector filter, but using regular expressions. It matches the specified dimension with the given pattern. The pattern can be any standard [Java regular expression](http://docs.oracle.com/javase/6/docs/api/java/util/regex/Pattern.html).
|
||||||
|
@ -24,6 +26,9 @@ The regular expression filter is similar to the selector filter, but using regul
|
||||||
"filter": { "type": "regex", "dimension": <dimension_string>, "pattern": <pattern_string> }
|
"filter": { "type": "regex", "dimension": <dimension_string>, "pattern": <pattern_string> }
|
||||||
```
|
```
|
||||||
|
|
||||||
|
The regex filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
||||||
|
|
||||||
|
|
||||||
### Logical expression filters
|
### Logical expression filters
|
||||||
|
|
||||||
#### AND
|
#### AND
|
||||||
|
@ -81,11 +86,19 @@ The following matches any dimension values for the dimension `name` between `'ba
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
|
||||||
|
The JavaScript filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
||||||
|
|
||||||
|
|
||||||
### Extraction filter
|
### Extraction filter
|
||||||
|
|
||||||
|
<div class="note caution">
|
||||||
|
The extraction filter is now deprecated. The selector filter with an extraction function specified
|
||||||
|
provides identical functionality and should be used instead.
|
||||||
|
</div>
|
||||||
|
|
||||||
Extraction filter matches a dimension using some specific [Extraction function](./dimensionspecs.html#extraction-functions).
|
Extraction filter matches a dimension using some specific [Extraction function](./dimensionspecs.html#extraction-functions).
|
||||||
The following filter matches the values for which the extraction function has transformation entry `input_key=output_value` where
|
The following filter matches the values for which the extraction function has transformation entry `input_key=output_value` where
|
||||||
`output_value` is equal to the filter `value` and `input_key` is present as dimension.
|
`output_value` is equal to the filter `value` and `input_key` is present as dimension.
|
||||||
|
|
||||||
**Example**
|
**Example**
|
||||||
The following matches dimension values in `[product_1, product_3, product_5]` for the column `product`
|
The following matches dimension values in `[product_1, product_3, product_5]` for the column `product`
|
||||||
|
@ -110,6 +123,7 @@ The following matches dimension values in `[product_1, product_3, product_5]` fo
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
|
||||||
### Search filter
|
### Search filter
|
||||||
|
|
||||||
Search filters can be used to filter on partial string matches.
|
Search filters can be used to filter on partial string matches.
|
||||||
|
@ -132,6 +146,10 @@ Search filters can be used to filter on partial string matches.
|
||||||
|type|This String should always be "search".|yes|
|
|type|This String should always be "search".|yes|
|
||||||
|dimension|The dimension to perform the search over.|yes|
|
|dimension|The dimension to perform the search over.|yes|
|
||||||
|query|A JSON object for the type of search. See below for more information.|yes|
|
|query|A JSON object for the type of search. See below for more information.|yes|
|
||||||
|
|extractionFn|[Extraction function](#filtering-with-extraction-functions) to apply to the dimension|no|
|
||||||
|
|
||||||
|
The search filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
||||||
|
|
||||||
|
|
||||||
### In filter
|
### In filter
|
||||||
|
|
||||||
|
@ -151,6 +169,9 @@ The grammar for a IN filter is as follows:
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
|
||||||
|
The IN filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
||||||
|
|
||||||
|
|
||||||
### Bound filter
|
### Bound filter
|
||||||
|
|
||||||
Bound filter can be used to filter by comparing dimension values to an upper value or/and a lower value.
|
Bound filter can be used to filter by comparing dimension values to an upper value or/and a lower value.
|
||||||
|
@ -158,6 +179,8 @@ By default Comparison is string based and **case sensitive**.
|
||||||
To use numeric comparison you can set `alphaNumeric` to `true`.
|
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`.
|
By default the bound filter is a not a strict inclusion `inputString <= upper && inputSting >= lower`.
|
||||||
|
|
||||||
|
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 grammar for a bound filter is as follows:
|
||||||
|
|
||||||
```json
|
```json
|
||||||
|
@ -246,6 +269,8 @@ For instance suppose lower bound is `100` and value is `10K` the filter will mat
|
||||||
Now suppose that the lower bound is `110` the filter will not match (`110 < 10K` returns `false`)
|
Now suppose that the lower bound is `110` the filter will not match (`110 < 10K` returns `false`)
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
#### Search Query Spec
|
#### Search Query Spec
|
||||||
|
|
||||||
##### Insensitive Contains
|
##### Insensitive Contains
|
||||||
|
@ -270,3 +295,38 @@ Now suppose that the lower bound is `110` the filter will not match (`110 < 10K`
|
||||||
|type|This String should always be "contains".|yes|
|
|type|This String should always be "contains".|yes|
|
||||||
|value|A String value to run the search over.|yes|
|
|value|A String value to run the search over.|yes|
|
||||||
|caseSensitive|Whether two string should be compared as case sensitive or not|yes|
|
|caseSensitive|Whether two string should be compared as case sensitive or not|yes|
|
||||||
|
|
||||||
|
|
||||||
|
### Filtering with Extraction Functions
|
||||||
|
Some filters optionally support the use of extraction functions.
|
||||||
|
An extraction function is defined by setting the "extractionFn" field on a filter.
|
||||||
|
See [Extraction function](./dimensionspecs.html#extraction-functions) for more details on extraction functions.
|
||||||
|
|
||||||
|
If specified, the extraction function will be used to transform input values before the filter is applied.
|
||||||
|
The example below shows a selector filter combined with an extraction function. This filter will transform input values
|
||||||
|
according to the values defined in the lookup map; transformed values will then be matched with the string "bar_1".
|
||||||
|
|
||||||
|
|
||||||
|
**Example**
|
||||||
|
The following matches dimension values in `[product_1, product_3, product_5]` for the column `product`
|
||||||
|
|
||||||
|
```json
|
||||||
|
{
|
||||||
|
"filter": {
|
||||||
|
"type": "selector",
|
||||||
|
"dimension": "product",
|
||||||
|
"value": "bar_1",
|
||||||
|
"extractionFn": {
|
||||||
|
"type": "lookup",
|
||||||
|
"lookup": {
|
||||||
|
"type": "map",
|
||||||
|
"map": {
|
||||||
|
"product_1": "bar_1",
|
||||||
|
"product_5": "bar_1",
|
||||||
|
"product_3": "bar_1"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
```
|
|
@ -48,7 +48,7 @@ public class DatasourceIngestionSpecTest
|
||||||
interval,
|
interval,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new SelectorDimFilter("dim", "value"),
|
new SelectorDimFilter("dim", "value", null),
|
||||||
QueryGranularity.DAY,
|
QueryGranularity.DAY,
|
||||||
Lists.newArrayList("d1", "d2"),
|
Lists.newArrayList("d1", "d2"),
|
||||||
Lists.newArrayList("m1", "m2", "m3"),
|
Lists.newArrayList("m1", "m2", "m3"),
|
||||||
|
@ -132,7 +132,7 @@ public class DatasourceIngestionSpecTest
|
||||||
128
|
128
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
new SelectorDimFilter("dim", "value"),
|
new SelectorDimFilter("dim", "value", null),
|
||||||
QueryGranularity.DAY,
|
QueryGranularity.DAY,
|
||||||
Lists.newArrayList("d1", "d2"),
|
Lists.newArrayList("d1", "d2"),
|
||||||
Lists.newArrayList("m1", "m2", "m3"),
|
Lists.newArrayList("m1", "m2", "m3"),
|
||||||
|
|
|
@ -305,7 +305,7 @@ public class IngestSegmentFirehoseFactoryTest
|
||||||
new IngestSegmentFirehoseFactory(
|
new IngestSegmentFirehoseFactory(
|
||||||
DATA_SOURCE_NAME,
|
DATA_SOURCE_NAME,
|
||||||
FOREVER,
|
FOREVER,
|
||||||
new SelectorDimFilter(DIM_NAME, DIM_VALUE),
|
new SelectorDimFilter(DIM_NAME, DIM_VALUE, null),
|
||||||
dim_names,
|
dim_names,
|
||||||
metric_names,
|
metric_names,
|
||||||
Guice.createInjector(
|
Guice.createInjector(
|
||||||
|
|
|
@ -163,9 +163,9 @@ public class Druids
|
||||||
|
|
||||||
public OrDimFilterBuilder fields(String dimensionName, String value, String... values)
|
public OrDimFilterBuilder fields(String dimensionName, String value, String... values)
|
||||||
{
|
{
|
||||||
fields = Lists.<DimFilter>newArrayList(new SelectorDimFilter(dimensionName, value));
|
fields = Lists.<DimFilter>newArrayList(new SelectorDimFilter(dimensionName, value, null));
|
||||||
for (String val : values) {
|
for (String val : values) {
|
||||||
fields.add(new SelectorDimFilter(dimensionName, val));
|
fields.add(new SelectorDimFilter(dimensionName, val, null));
|
||||||
}
|
}
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
@ -256,7 +256,7 @@ public class Druids
|
||||||
|
|
||||||
public SelectorDimFilter build()
|
public SelectorDimFilter build()
|
||||||
{
|
{
|
||||||
return new SelectorDimFilter(dimension, value);
|
return new SelectorDimFilter(dimension, value, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
public SelectorDimFilterBuilder copy(SelectorDimFilterBuilder builder)
|
public SelectorDimFilterBuilder copy(SelectorDimFilterBuilder builder)
|
||||||
|
@ -459,13 +459,13 @@ public class Druids
|
||||||
|
|
||||||
public TimeseriesQueryBuilder filters(String dimensionName, String value)
|
public TimeseriesQueryBuilder filters(String dimensionName, String value)
|
||||||
{
|
{
|
||||||
dimFilter = new SelectorDimFilter(dimensionName, value);
|
dimFilter = new SelectorDimFilter(dimensionName, value, null);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public TimeseriesQueryBuilder filters(String dimensionName, String value, String... values)
|
public TimeseriesQueryBuilder filters(String dimensionName, String value, String... values)
|
||||||
{
|
{
|
||||||
dimFilter = new InDimFilter(dimensionName, Lists.asList(value, values));
|
dimFilter = new InDimFilter(dimensionName, Lists.asList(value, values), null);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -615,13 +615,13 @@ public class Druids
|
||||||
|
|
||||||
public SearchQueryBuilder filters(String dimensionName, String value)
|
public SearchQueryBuilder filters(String dimensionName, String value)
|
||||||
{
|
{
|
||||||
dimFilter = new SelectorDimFilter(dimensionName, value);
|
dimFilter = new SelectorDimFilter(dimensionName, value, null);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public SearchQueryBuilder filters(String dimensionName, String value, String... values)
|
public SearchQueryBuilder filters(String dimensionName, String value, String... values)
|
||||||
{
|
{
|
||||||
dimFilter = new InDimFilter(dimensionName, Lists.asList(value, values));
|
dimFilter = new InDimFilter(dimensionName, Lists.asList(value, values), null);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1159,13 +1159,13 @@ public class Druids
|
||||||
|
|
||||||
public SelectQueryBuilder filters(String dimensionName, String value)
|
public SelectQueryBuilder filters(String dimensionName, String value)
|
||||||
{
|
{
|
||||||
dimFilter = new SelectorDimFilter(dimensionName, value);
|
dimFilter = new SelectorDimFilter(dimensionName, value, null);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public SelectQueryBuilder filters(String dimensionName, String value, String... values)
|
public SelectQueryBuilder filters(String dimensionName, String value, String... values)
|
||||||
{
|
{
|
||||||
dimFilter = new InDimFilter(dimensionName, Lists.asList(value, values));
|
dimFilter = new InDimFilter(dimensionName, Lists.asList(value, values), null);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.metamx.common.StringUtils;
|
import com.metamx.common.StringUtils;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.segment.filter.BoundFilter;
|
import io.druid.segment.filter.BoundFilter;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -35,6 +36,7 @@ public class BoundDimFilter implements DimFilter
|
||||||
private final boolean lowerStrict;
|
private final boolean lowerStrict;
|
||||||
private final boolean upperStrict;
|
private final boolean upperStrict;
|
||||||
private final boolean alphaNumeric;
|
private final boolean alphaNumeric;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public BoundDimFilter(
|
public BoundDimFilter(
|
||||||
|
@ -43,7 +45,8 @@ public class BoundDimFilter implements DimFilter
|
||||||
@JsonProperty("upper") String upper,
|
@JsonProperty("upper") String upper,
|
||||||
@JsonProperty("lowerStrict") Boolean lowerStrict,
|
@JsonProperty("lowerStrict") Boolean lowerStrict,
|
||||||
@JsonProperty("upperStrict") Boolean upperStrict,
|
@JsonProperty("upperStrict") Boolean upperStrict,
|
||||||
@JsonProperty("alphaNumeric") Boolean alphaNumeric
|
@JsonProperty("alphaNumeric") Boolean alphaNumeric,
|
||||||
|
@JsonProperty("extractionFn") ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimension = Preconditions.checkNotNull(dimension, "dimension can not be null");
|
this.dimension = Preconditions.checkNotNull(dimension, "dimension can not be null");
|
||||||
|
@ -53,6 +56,7 @@ public class BoundDimFilter implements DimFilter
|
||||||
this.lowerStrict = (lowerStrict == null) ? false : lowerStrict;
|
this.lowerStrict = (lowerStrict == null) ? false : lowerStrict;
|
||||||
this.upperStrict = (upperStrict == null) ? false : upperStrict;
|
this.upperStrict = (upperStrict == null) ? false : upperStrict;
|
||||||
this.alphaNumeric = (alphaNumeric == null) ? false : alphaNumeric;
|
this.alphaNumeric = (alphaNumeric == null) ? false : alphaNumeric;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -101,6 +105,12 @@ public class BoundDimFilter implements DimFilter
|
||||||
return upper != null;
|
return upper != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public ExtractionFn getExtractionFn()
|
||||||
|
{
|
||||||
|
return extractionFn;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] getCacheKey()
|
public byte[] getCacheKey()
|
||||||
{
|
{
|
||||||
|
@ -118,11 +128,14 @@ public class BoundDimFilter implements DimFilter
|
||||||
byte upperStrictByte = (this.isUpperStrict() == false) ? 0x0 : (byte) 1;
|
byte upperStrictByte = (this.isUpperStrict() == false) ? 0x0 : (byte) 1;
|
||||||
byte AlphaNumericByte = (this.isAlphaNumeric() == false) ? 0x0 : (byte) 1;
|
byte AlphaNumericByte = (this.isAlphaNumeric() == false) ? 0x0 : (byte) 1;
|
||||||
|
|
||||||
|
byte[] extractionFnBytes = extractionFn == null ? new byte[0] : extractionFn.getCacheKey();
|
||||||
|
|
||||||
ByteBuffer boundCacheBuffer = ByteBuffer.allocate(
|
ByteBuffer boundCacheBuffer = ByteBuffer.allocate(
|
||||||
8
|
9
|
||||||
+ dimensionBytes.length
|
+ dimensionBytes.length
|
||||||
+ upperBytes.length
|
+ upperBytes.length
|
||||||
+ lowerBytes.length
|
+ lowerBytes.length
|
||||||
|
+ extractionFnBytes.length
|
||||||
);
|
);
|
||||||
boundCacheBuffer.put(DimFilterCacheHelper.BOUND_CACHE_ID)
|
boundCacheBuffer.put(DimFilterCacheHelper.BOUND_CACHE_ID)
|
||||||
.put(boundType)
|
.put(boundType)
|
||||||
|
@ -134,7 +147,9 @@ public class BoundDimFilter implements DimFilter
|
||||||
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
.put(upperBytes)
|
.put(upperBytes)
|
||||||
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
.put(lowerBytes);
|
.put(lowerBytes)
|
||||||
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
|
.put(extractionFnBytes);
|
||||||
return boundCacheBuffer.array();
|
return boundCacheBuffer.array();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -156,7 +171,7 @@ public class BoundDimFilter implements DimFilter
|
||||||
if (this == o) {
|
if (this == o) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
if (!(o instanceof BoundDimFilter)) {
|
if (o == null || getClass() != o.getClass()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -177,7 +192,12 @@ public class BoundDimFilter implements DimFilter
|
||||||
if (getUpper() != null ? !getUpper().equals(that.getUpper()) : that.getUpper() != null) {
|
if (getUpper() != null ? !getUpper().equals(that.getUpper()) : that.getUpper() != null) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
return !(getLower() != null ? !getLower().equals(that.getLower()) : that.getLower() != null);
|
if (getLower() != null ? !getLower().equals(that.getLower()) : that.getLower() != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return getExtractionFn() != null
|
||||||
|
? getExtractionFn().equals(that.getExtractionFn())
|
||||||
|
: that.getExtractionFn() == null;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -190,6 +210,7 @@ public class BoundDimFilter implements DimFilter
|
||||||
result = 31 * result + (isLowerStrict() ? 1 : 0);
|
result = 31 * result + (isLowerStrict() ? 1 : 0);
|
||||||
result = 31 * result + (isUpperStrict() ? 1 : 0);
|
result = 31 * result + (isUpperStrict() ? 1 : 0);
|
||||||
result = 31 * result + (isAlphaNumeric() ? 1 : 0);
|
result = 31 * result + (isAlphaNumeric() ? 1 : 0);
|
||||||
|
result = 31 * result + (getExtractionFn() != null ? getExtractionFn().hashCode() : 0);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,7 +34,7 @@ public class DimFilters
|
||||||
{
|
{
|
||||||
public static SelectorDimFilter dimEquals(String dimension, String value)
|
public static SelectorDimFilter dimEquals(String dimension, String value)
|
||||||
{
|
{
|
||||||
return new SelectorDimFilter(dimension, value);
|
return new SelectorDimFilter(dimension, value, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static AndDimFilter and(DimFilter... filters)
|
public static AndDimFilter and(DimFilter... filters)
|
||||||
|
@ -64,7 +64,7 @@ public class DimFilters
|
||||||
|
|
||||||
public static RegexDimFilter regex(String dimension, String pattern)
|
public static RegexDimFilter regex(String dimension, String pattern)
|
||||||
{
|
{
|
||||||
return new RegexDimFilter(dimension, pattern);
|
return new RegexDimFilter(dimension, pattern, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static DimFilter dimEquals(final String dimension, String... values)
|
public static DimFilter dimEquals(final String dimension, String... values)
|
||||||
|
|
|
@ -22,17 +22,21 @@ package io.druid.query.filter;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Strings;
|
||||||
import com.metamx.common.StringUtils;
|
import com.metamx.common.StringUtils;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.lookup.LookupExtractionFn;
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
import io.druid.query.lookup.LookupExtractor;
|
import io.druid.query.lookup.LookupExtractor;
|
||||||
import io.druid.segment.filter.ExtractionFilter;
|
import io.druid.segment.filter.SelectorFilter;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* This class is deprecated, use SelectorDimFilter instead: {@link io.druid.query.filter.SelectorDimFilter}
|
||||||
*/
|
*/
|
||||||
|
@Deprecated
|
||||||
public class ExtractionDimFilter implements DimFilter
|
public class ExtractionDimFilter implements DimFilter
|
||||||
{
|
{
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
|
@ -96,22 +100,13 @@ public class ExtractionDimFilter implements DimFilter
|
||||||
@Override
|
@Override
|
||||||
public DimFilter optimize()
|
public DimFilter optimize()
|
||||||
{
|
{
|
||||||
if (this.getExtractionFn() instanceof LookupExtractionFn
|
return new SelectorDimFilter(dimension, value, extractionFn).optimize();
|
||||||
&& ((LookupExtractionFn) this.getExtractionFn()).isOptimize()) {
|
|
||||||
LookupExtractor lookup = ((LookupExtractionFn) this.getExtractionFn()).getLookup();
|
|
||||||
final List<String> keys = lookup.unapply(this.getValue());
|
|
||||||
final String dimensionName = this.getDimension();
|
|
||||||
if (!keys.isEmpty()) {
|
|
||||||
return new InDimFilter(dimensionName, keys);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return this;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Filter toFilter()
|
public Filter toFilter()
|
||||||
{
|
{
|
||||||
return new ExtractionFilter(dimension, value, extractionFn);
|
return new SelectorFilter(dimension, value, extractionFn);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -29,18 +29,28 @@ import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.common.StringUtils;
|
import com.metamx.common.StringUtils;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
|
import io.druid.query.lookup.LookupExtractor;
|
||||||
import io.druid.segment.filter.InFilter;
|
import io.druid.segment.filter.InFilter;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
public class InDimFilter implements DimFilter
|
public class InDimFilter implements DimFilter
|
||||||
{
|
{
|
||||||
private final List<String> values;
|
private final List<String> values;
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public InDimFilter(@JsonProperty("dimension") String dimension, @JsonProperty("values") List<String> values)
|
public InDimFilter(
|
||||||
|
@JsonProperty("dimension") String dimension,
|
||||||
|
@JsonProperty("values") List<String> values,
|
||||||
|
@JsonProperty("extractionFn") ExtractionFn extractionFn
|
||||||
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(dimension, "dimension can not be null");
|
Preconditions.checkNotNull(dimension, "dimension can not be null");
|
||||||
Preconditions.checkArgument(values != null && !values.isEmpty(), "values can not be null or empty");
|
Preconditions.checkArgument(values != null && !values.isEmpty(), "values can not be null or empty");
|
||||||
|
@ -58,6 +68,7 @@ public class InDimFilter implements DimFilter
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -72,6 +83,12 @@ public class InDimFilter implements DimFilter
|
||||||
return values;
|
return values;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public ExtractionFn getExtractionFn()
|
||||||
|
{
|
||||||
|
return extractionFn;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] getCacheKey()
|
public byte[] getCacheKey()
|
||||||
{
|
{
|
||||||
|
@ -84,10 +101,13 @@ public class InDimFilter implements DimFilter
|
||||||
valuesBytesSize += valuesBytes[index].length + 1;
|
valuesBytesSize += valuesBytes[index].length + 1;
|
||||||
++index;
|
++index;
|
||||||
}
|
}
|
||||||
|
byte[] extractionFnBytes = extractionFn == null ? new byte[0] : extractionFn.getCacheKey();
|
||||||
|
|
||||||
ByteBuffer filterCacheKey = ByteBuffer.allocate(2 + dimensionBytes.length + valuesBytesSize)
|
ByteBuffer filterCacheKey = ByteBuffer.allocate(3 + dimensionBytes.length + valuesBytesSize + extractionFnBytes.length)
|
||||||
.put(DimFilterCacheHelper.IN_CACHE_ID)
|
.put(DimFilterCacheHelper.IN_CACHE_ID)
|
||||||
.put(dimensionBytes)
|
.put(dimensionBytes)
|
||||||
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
|
.put(extractionFnBytes)
|
||||||
.put(DimFilterCacheHelper.STRING_SEPARATOR);
|
.put(DimFilterCacheHelper.STRING_SEPARATOR);
|
||||||
for (byte[] bytes : valuesBytes) {
|
for (byte[] bytes : valuesBytes) {
|
||||||
filterCacheKey.put(bytes)
|
filterCacheKey.put(bytes)
|
||||||
|
@ -105,15 +125,7 @@ public class InDimFilter implements DimFilter
|
||||||
@Override
|
@Override
|
||||||
public Filter toFilter()
|
public Filter toFilter()
|
||||||
{
|
{
|
||||||
return new InFilter(dimension, ImmutableSet.copyOf(values));
|
return new InFilter(dimension, ImmutableSet.copyOf(values), extractionFn);
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode()
|
|
||||||
{
|
|
||||||
int result = getValues().hashCode();
|
|
||||||
result = 31 * result + getDimension().hashCode();
|
|
||||||
return result;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -122,16 +134,28 @@ public class InDimFilter implements DimFilter
|
||||||
if (this == o) {
|
if (this == o) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
if (!(o instanceof InDimFilter)) {
|
if (o == null || getClass() != o.getClass()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
InDimFilter that = (InDimFilter) o;
|
InDimFilter that = (InDimFilter) o;
|
||||||
|
|
||||||
if (!values.equals(that.values)) {
|
if (values != null ? !values.equals(that.values) : that.values != null) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
return dimension.equals(that.dimension);
|
if (!dimension.equals(that.dimension)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return extractionFn != null ? extractionFn.equals(that.extractionFn) : that.extractionFn == null;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
int result = values != null ? values.hashCode() : 0;
|
||||||
|
result = 31 * result + dimension.hashCode();
|
||||||
|
result = 31 * result + (extractionFn != null ? extractionFn.hashCode() : 0);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.metamx.common.StringUtils;
|
import com.metamx.common.StringUtils;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.segment.filter.JavaScriptFilter;
|
import io.druid.segment.filter.JavaScriptFilter;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -31,17 +32,20 @@ public class JavaScriptDimFilter implements DimFilter
|
||||||
{
|
{
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
private final String function;
|
private final String function;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public JavaScriptDimFilter(
|
public JavaScriptDimFilter(
|
||||||
@JsonProperty("dimension") String dimension,
|
@JsonProperty("dimension") String dimension,
|
||||||
@JsonProperty("function") String function
|
@JsonProperty("function") String function,
|
||||||
|
@JsonProperty("extractionFn") ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||||
Preconditions.checkArgument(function != null, "function must not be null");
|
Preconditions.checkArgument(function != null, "function must not be null");
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.function = function;
|
this.function = function;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -56,17 +60,26 @@ public class JavaScriptDimFilter implements DimFilter
|
||||||
return function;
|
return function;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public ExtractionFn getExtractionFn()
|
||||||
|
{
|
||||||
|
return extractionFn;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] getCacheKey()
|
public byte[] getCacheKey()
|
||||||
{
|
{
|
||||||
final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
|
final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
|
||||||
final byte[] functionBytes = StringUtils.toUtf8(function);
|
final byte[] functionBytes = StringUtils.toUtf8(function);
|
||||||
|
byte[] extractionFnBytes = extractionFn == null ? new byte[0] : extractionFn.getCacheKey();
|
||||||
|
|
||||||
return ByteBuffer.allocate(2 + dimensionBytes.length + functionBytes.length)
|
return ByteBuffer.allocate(3 + dimensionBytes.length + functionBytes.length + extractionFnBytes.length)
|
||||||
.put(DimFilterCacheHelper.JAVASCRIPT_CACHE_ID)
|
.put(DimFilterCacheHelper.JAVASCRIPT_CACHE_ID)
|
||||||
.put(dimensionBytes)
|
.put(dimensionBytes)
|
||||||
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
.put(functionBytes)
|
.put(functionBytes)
|
||||||
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
|
.put(extractionFnBytes)
|
||||||
.array();
|
.array();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -79,7 +92,7 @@ public class JavaScriptDimFilter implements DimFilter
|
||||||
@Override
|
@Override
|
||||||
public Filter toFilter()
|
public Filter toFilter()
|
||||||
{
|
{
|
||||||
return new JavaScriptFilter(dimension, function);
|
return new JavaScriptFilter(dimension, function, extractionFn);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -88,6 +101,7 @@ public class JavaScriptDimFilter implements DimFilter
|
||||||
return "JavaScriptDimFilter{" +
|
return "JavaScriptDimFilter{" +
|
||||||
"dimension='" + dimension + '\'' +
|
"dimension='" + dimension + '\'' +
|
||||||
", function='" + function + '\'' +
|
", function='" + function + '\'' +
|
||||||
|
", extractionFn='" + extractionFn + '\'' +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.metamx.common.StringUtils;
|
import com.metamx.common.StringUtils;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.segment.filter.RegexFilter;
|
import io.druid.segment.filter.RegexFilter;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -33,17 +34,20 @@ public class RegexDimFilter implements DimFilter
|
||||||
{
|
{
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
private final String pattern;
|
private final String pattern;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public RegexDimFilter(
|
public RegexDimFilter(
|
||||||
@JsonProperty("dimension") String dimension,
|
@JsonProperty("dimension") String dimension,
|
||||||
@JsonProperty("pattern") String pattern
|
@JsonProperty("pattern") String pattern,
|
||||||
|
@JsonProperty("extractionFn") ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||||
Preconditions.checkArgument(pattern != null, "pattern must not be null");
|
Preconditions.checkArgument(pattern != null, "pattern must not be null");
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.pattern = pattern;
|
this.pattern = pattern;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -58,17 +62,26 @@ public class RegexDimFilter implements DimFilter
|
||||||
return pattern;
|
return pattern;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public ExtractionFn getExtractionFn()
|
||||||
|
{
|
||||||
|
return extractionFn;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] getCacheKey()
|
public byte[] getCacheKey()
|
||||||
{
|
{
|
||||||
final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
|
final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
|
||||||
final byte[] patternBytes = StringUtils.toUtf8(pattern);
|
final byte[] patternBytes = StringUtils.toUtf8(pattern);
|
||||||
|
byte[] extractionFnBytes = extractionFn == null ? new byte[0] : extractionFn.getCacheKey();
|
||||||
|
|
||||||
return ByteBuffer.allocate(2 + dimensionBytes.length + patternBytes.length)
|
return ByteBuffer.allocate(3 + dimensionBytes.length + patternBytes.length + extractionFnBytes.length)
|
||||||
.put(DimFilterCacheHelper.REGEX_CACHE_ID)
|
.put(DimFilterCacheHelper.REGEX_CACHE_ID)
|
||||||
.put(dimensionBytes)
|
.put(dimensionBytes)
|
||||||
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
.put(patternBytes)
|
.put(patternBytes)
|
||||||
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
|
.put(extractionFnBytes)
|
||||||
.array();
|
.array();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -81,7 +94,7 @@ public class RegexDimFilter implements DimFilter
|
||||||
@Override
|
@Override
|
||||||
public Filter toFilter()
|
public Filter toFilter()
|
||||||
{
|
{
|
||||||
return new RegexFilter(dimension, pattern);
|
return new RegexFilter(dimension, pattern, extractionFn);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -90,6 +103,7 @@ public class RegexDimFilter implements DimFilter
|
||||||
return "RegexDimFilter{" +
|
return "RegexDimFilter{" +
|
||||||
"dimension='" + dimension + '\'' +
|
"dimension='" + dimension + '\'' +
|
||||||
", pattern='" + pattern + '\'' +
|
", pattern='" + pattern + '\'' +
|
||||||
|
", extractionFn='" + extractionFn + '\'' +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.query.filter;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.metamx.common.StringUtils;
|
import com.metamx.common.StringUtils;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.search.search.SearchQuerySpec;
|
import io.druid.query.search.search.SearchQuerySpec;
|
||||||
import io.druid.segment.filter.SearchQueryFilter;
|
import io.druid.segment.filter.SearchQueryFilter;
|
||||||
|
|
||||||
|
@ -33,10 +34,12 @@ public class SearchQueryDimFilter implements DimFilter
|
||||||
{
|
{
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
private final SearchQuerySpec query;
|
private final SearchQuerySpec query;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
public SearchQueryDimFilter(
|
public SearchQueryDimFilter(
|
||||||
@JsonProperty("dimension") String dimension,
|
@JsonProperty("dimension") String dimension,
|
||||||
@JsonProperty("query") SearchQuerySpec query
|
@JsonProperty("query") SearchQuerySpec query,
|
||||||
|
@JsonProperty("extractionFn") ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||||
|
@ -44,6 +47,7 @@ public class SearchQueryDimFilter implements DimFilter
|
||||||
|
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.query = query;
|
this.query = query;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -58,17 +62,26 @@ public class SearchQueryDimFilter implements DimFilter
|
||||||
return query;
|
return query;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public ExtractionFn getExtractionFn()
|
||||||
|
{
|
||||||
|
return extractionFn;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] getCacheKey()
|
public byte[] getCacheKey()
|
||||||
{
|
{
|
||||||
final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
|
final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
|
||||||
final byte[] queryBytes = query.getCacheKey();
|
final byte[] queryBytes = query.getCacheKey();
|
||||||
|
byte[] extractionFnBytes = extractionFn == null ? new byte[0] : extractionFn.getCacheKey();
|
||||||
|
|
||||||
return ByteBuffer.allocate(2 + dimensionBytes.length + queryBytes.length)
|
return ByteBuffer.allocate(3 + dimensionBytes.length + queryBytes.length + extractionFnBytes.length)
|
||||||
.put(DimFilterCacheHelper.SEARCH_QUERY_TYPE_ID)
|
.put(DimFilterCacheHelper.SEARCH_QUERY_TYPE_ID)
|
||||||
.put(dimensionBytes)
|
.put(dimensionBytes)
|
||||||
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
.put(queryBytes)
|
.put(queryBytes)
|
||||||
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
|
.put(extractionFnBytes)
|
||||||
.array();
|
.array();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -81,7 +94,7 @@ public class SearchQueryDimFilter implements DimFilter
|
||||||
@Override
|
@Override
|
||||||
public Filter toFilter()
|
public Filter toFilter()
|
||||||
{
|
{
|
||||||
return new SearchQueryFilter(dimension, query);
|
return new SearchQueryFilter(dimension, query, extractionFn);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -90,6 +103,7 @@ public class SearchQueryDimFilter implements DimFilter
|
||||||
return "SearchQueryDimFilter{" +
|
return "SearchQueryDimFilter{" +
|
||||||
"dimension='" + dimension + '\'' +
|
"dimension='" + dimension + '\'' +
|
||||||
", query=" + query +
|
", query=" + query +
|
||||||
|
", extractionFn='" + extractionFn + '\'' +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,11 +21,21 @@ package io.druid.query.filter;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Strings;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
import com.metamx.common.StringUtils;
|
import com.metamx.common.StringUtils;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
|
import io.druid.query.lookup.LookupExtractor;
|
||||||
import io.druid.segment.filter.SelectorFilter;
|
import io.druid.segment.filter.SelectorFilter;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -33,17 +43,20 @@ public class SelectorDimFilter implements DimFilter
|
||||||
{
|
{
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
private final String value;
|
private final String value;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public SelectorDimFilter(
|
public SelectorDimFilter(
|
||||||
@JsonProperty("dimension") String dimension,
|
@JsonProperty("dimension") String dimension,
|
||||||
@JsonProperty("value") String value
|
@JsonProperty("value") String value,
|
||||||
|
@JsonProperty("extractionFn") ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||||
|
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -51,25 +64,62 @@ public class SelectorDimFilter implements DimFilter
|
||||||
{
|
{
|
||||||
byte[] dimensionBytes = StringUtils.toUtf8(dimension);
|
byte[] dimensionBytes = StringUtils.toUtf8(dimension);
|
||||||
byte[] valueBytes = (value == null) ? new byte[]{} : StringUtils.toUtf8(value);
|
byte[] valueBytes = (value == null) ? new byte[]{} : StringUtils.toUtf8(value);
|
||||||
|
byte[] extractionFnBytes = extractionFn == null ? new byte[0] : extractionFn.getCacheKey();
|
||||||
|
|
||||||
return ByteBuffer.allocate(2 + dimensionBytes.length + valueBytes.length)
|
return ByteBuffer.allocate(3 + dimensionBytes.length + valueBytes.length + extractionFnBytes.length)
|
||||||
.put(DimFilterCacheHelper.SELECTOR_CACHE_ID)
|
.put(DimFilterCacheHelper.SELECTOR_CACHE_ID)
|
||||||
.put(dimensionBytes)
|
.put(dimensionBytes)
|
||||||
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
.put(valueBytes)
|
.put(valueBytes)
|
||||||
|
.put(DimFilterCacheHelper.STRING_SEPARATOR)
|
||||||
|
.put(extractionFnBytes)
|
||||||
.array();
|
.array();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimFilter optimize()
|
public DimFilter optimize()
|
||||||
{
|
{
|
||||||
|
if (this.getExtractionFn() instanceof LookupExtractionFn
|
||||||
|
&& ((LookupExtractionFn) this.getExtractionFn()).isOptimize()) {
|
||||||
|
LookupExtractionFn exFn = (LookupExtractionFn) this.getExtractionFn();
|
||||||
|
LookupExtractor lookup = exFn.getLookup();
|
||||||
|
|
||||||
|
final String convertedValue = Strings.emptyToNull(value);
|
||||||
|
|
||||||
|
// We cannot do an unapply()-based optimization if the selector value
|
||||||
|
// and the replaceMissingValuesWith value are the same, since we have to match on
|
||||||
|
// all values that are not present in the lookup.
|
||||||
|
if (!exFn.isRetainMissingValue() && Objects.equals(convertedValue, exFn.getReplaceMissingValueWith())) {
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
final String mappingForValue = lookup.apply(convertedValue);
|
||||||
|
final List<String> keys = new ArrayList<>();
|
||||||
|
keys.addAll(lookup.unapply(convertedValue));
|
||||||
|
|
||||||
|
// If retainMissingValues is true and the selector value is not in the lookup map,
|
||||||
|
// there may be row values that match the selector value but are not included
|
||||||
|
// in the lookup map. Match on the selector value as well.
|
||||||
|
// If the selector value is overwritten in the lookup map, don't add selector value to keys.
|
||||||
|
if (exFn.isRetainMissingValue() && mappingForValue == null) {
|
||||||
|
keys.add(convertedValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (keys.isEmpty()) {
|
||||||
|
return this;
|
||||||
|
} else if (keys.size() == 1) {
|
||||||
|
return new SelectorDimFilter(dimension, keys.get(0), null);
|
||||||
|
} else {
|
||||||
|
return new InDimFilter(dimension, keys, null);
|
||||||
|
}
|
||||||
|
}
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Filter toFilter()
|
public Filter toFilter()
|
||||||
{
|
{
|
||||||
return new SelectorFilter(dimension, value);
|
return new SelectorFilter(dimension, value, extractionFn);
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -84,6 +134,22 @@ public class SelectorDimFilter implements DimFilter
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public ExtractionFn getExtractionFn()
|
||||||
|
{
|
||||||
|
return extractionFn;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
if (extractionFn != null) {
|
||||||
|
return String.format("%s(%s) = %s", extractionFn, dimension, value);
|
||||||
|
} else {
|
||||||
|
return String.format("%s = %s", dimension, value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o)
|
public boolean equals(Object o)
|
||||||
{
|
{
|
||||||
|
@ -96,27 +162,21 @@ public class SelectorDimFilter implements DimFilter
|
||||||
|
|
||||||
SelectorDimFilter that = (SelectorDimFilter) o;
|
SelectorDimFilter that = (SelectorDimFilter) o;
|
||||||
|
|
||||||
if (dimension != null ? !dimension.equals(that.dimension) : that.dimension != null) {
|
if (!dimension.equals(that.dimension)) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
if (value != null ? !value.equals(that.value) : that.value != null) {
|
if (value != null ? !value.equals(that.value) : that.value != null) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
return extractionFn != null ? extractionFn.equals(that.extractionFn) : that.extractionFn == null;
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
int result = dimension != null ? dimension.hashCode() : 0;
|
int result = dimension.hashCode();
|
||||||
result = 31 * result + (value != null ? value.hashCode() : 0);
|
result = 31 * result + (value != null ? value.hashCode() : 0);
|
||||||
|
result = 31 * result + (extractionFn != null ? extractionFn.hashCode() : 0);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString()
|
|
||||||
{
|
|
||||||
return String.format("%s = %s", dimension, value);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -246,13 +246,13 @@ public class TopNQueryBuilder
|
||||||
|
|
||||||
public TopNQueryBuilder filters(String dimensionName, String value)
|
public TopNQueryBuilder filters(String dimensionName, String value)
|
||||||
{
|
{
|
||||||
dimFilter = new SelectorDimFilter(dimensionName, value);
|
dimFilter = new SelectorDimFilter(dimensionName, value, null);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public TopNQueryBuilder filters(String dimensionName, String value, String... values)
|
public TopNQueryBuilder filters(String dimensionName, String value, String... values)
|
||||||
{
|
{
|
||||||
dimFilter = new InDimFilter(dimensionName, Lists.asList(value, values));
|
dimFilter = new InDimFilter(dimensionName, Lists.asList(value, values), null);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -424,8 +424,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
{
|
{
|
||||||
final String value = column.lookupName(id);
|
final String value = column.lookupName(id);
|
||||||
return extractionFn == null ?
|
return extractionFn == null ?
|
||||||
Strings.nullToEmpty(value) :
|
value :
|
||||||
extractionFn.apply(Strings.nullToEmpty(value));
|
extractionFn.apply(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.segment.filter;
|
||||||
|
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.filter.BitmapIndexSelector;
|
import io.druid.query.filter.BitmapIndexSelector;
|
||||||
import io.druid.query.filter.BoundDimFilter;
|
import io.druid.query.filter.BoundDimFilter;
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
|
@ -37,6 +38,7 @@ public class BoundFilter implements Filter
|
||||||
{
|
{
|
||||||
private final BoundDimFilter boundDimFilter;
|
private final BoundDimFilter boundDimFilter;
|
||||||
private final Comparator<String> comparator;
|
private final Comparator<String> comparator;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
public BoundFilter(final BoundDimFilter boundDimFilter)
|
public BoundFilter(final BoundDimFilter boundDimFilter)
|
||||||
{
|
{
|
||||||
|
@ -44,6 +46,7 @@ public class BoundFilter implements Filter
|
||||||
this.comparator = boundDimFilter.isAlphaNumeric()
|
this.comparator = boundDimFilter.isAlphaNumeric()
|
||||||
? StringComparators.ALPHANUMERIC
|
? StringComparators.ALPHANUMERIC
|
||||||
: StringComparators.LEXICOGRAPHIC;
|
: StringComparators.LEXICOGRAPHIC;
|
||||||
|
this.extractionFn = boundDimFilter.getExtractionFn();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -60,7 +63,7 @@ public class BoundFilter implements Filter
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (boundDimFilter.isAlphaNumeric()) {
|
if (boundDimFilter.isAlphaNumeric() || extractionFn != null) {
|
||||||
// inspect all values
|
// inspect all values
|
||||||
|
|
||||||
// will be non-null because bitmapIndex was non-null
|
// will be non-null because bitmapIndex was non-null
|
||||||
|
@ -185,6 +188,10 @@ public class BoundFilter implements Filter
|
||||||
|
|
||||||
private boolean doesMatch(String input)
|
private boolean doesMatch(String input)
|
||||||
{
|
{
|
||||||
|
if (extractionFn != null) {
|
||||||
|
input = extractionFn.apply(input);
|
||||||
|
}
|
||||||
|
|
||||||
if (input == null) {
|
if (input == null) {
|
||||||
return (!boundDimFilter.hasLowerBound()
|
return (!boundDimFilter.hasLowerBound()
|
||||||
|| (boundDimFilter.getLower().isEmpty() && !boundDimFilter.isLowerStrict())) // lower bound allows null
|
|| (boundDimFilter.getLower().isEmpty() && !boundDimFilter.isLowerStrict())) // lower bound allows null
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.base.Function;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||||
import com.metamx.common.guava.FunctionalIterable;
|
import com.metamx.common.guava.FunctionalIterable;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.filter.BitmapIndexSelector;
|
import io.druid.query.filter.BitmapIndexSelector;
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
import io.druid.query.filter.ValueMatcher;
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
@ -37,27 +38,54 @@ class DimensionPredicateFilter implements Filter
|
||||||
{
|
{
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
private final Predicate<String> predicate;
|
private final Predicate<String> predicate;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
public DimensionPredicateFilter(
|
public DimensionPredicateFilter(
|
||||||
String dimension,
|
String dimension,
|
||||||
Predicate<String> predicate
|
Predicate<String> predicate,
|
||||||
|
ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.predicate = predicate;
|
this.predicate = predicate;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
|
public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
Indexed<String> dimValues = selector.getDimensionValues(dimension);
|
if (predicate == null) {
|
||||||
if (dimValues == null || dimValues.size() == 0 || predicate == null) {
|
|
||||||
return selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
return selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
||||||
}
|
}
|
||||||
|
Indexed<String> dimValues = selector.getDimensionValues(dimension);
|
||||||
|
if (dimValues == null || dimValues.size() == 0) {
|
||||||
|
boolean needsComplement = predicate.apply(extractionFn == null ? null : extractionFn.apply(null));
|
||||||
|
if (needsComplement) {
|
||||||
|
return selector.getBitmapFactory().complement(
|
||||||
|
selector.getBitmapFactory().makeEmptyImmutableBitmap(),
|
||||||
|
selector.getNumRows()
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
return selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
return selector.getBitmapFactory().union(
|
return selector.getBitmapFactory().union(
|
||||||
FunctionalIterable.create(dimValues)
|
FunctionalIterable.create(dimValues)
|
||||||
.filter(predicate)
|
.filter(
|
||||||
|
extractionFn == null ?
|
||||||
|
predicate
|
||||||
|
:
|
||||||
|
new Predicate<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean apply(@Nullable String input)
|
||||||
|
{
|
||||||
|
return predicate.apply(extractionFn.apply(input));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
)
|
||||||
.transform(
|
.transform(
|
||||||
new Function<String, ImmutableBitmap>()
|
new Function<String, ImmutableBitmap>()
|
||||||
{
|
{
|
||||||
|
@ -74,6 +102,18 @@ class DimensionPredicateFilter implements Filter
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
||||||
{
|
{
|
||||||
return factory.makeValueMatcher(dimension, predicate);
|
if (extractionFn == null) {
|
||||||
|
return factory.makeValueMatcher(dimension, predicate);
|
||||||
|
} else {
|
||||||
|
Predicate extractingPredicate = new Predicate()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean apply(@Nullable Object input)
|
||||||
|
{
|
||||||
|
return predicate.apply(extractionFn.apply(input));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
return factory.makeValueMatcher(dimension, extractingPredicate);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,119 +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.segment.filter;
|
|
||||||
|
|
||||||
import com.google.common.base.Predicate;
|
|
||||||
import com.google.common.base.Strings;
|
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
|
||||||
import io.druid.query.filter.BitmapIndexSelector;
|
|
||||||
import io.druid.query.filter.Filter;
|
|
||||||
import io.druid.query.filter.ValueMatcher;
|
|
||||||
import io.druid.query.filter.ValueMatcherFactory;
|
|
||||||
import io.druid.segment.data.Indexed;
|
|
||||||
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
/**
|
|
||||||
*/
|
|
||||||
public class ExtractionFilter implements Filter
|
|
||||||
{
|
|
||||||
private final String dimension;
|
|
||||||
private final String value;
|
|
||||||
private final ExtractionFn fn;
|
|
||||||
|
|
||||||
public ExtractionFilter(String dimension, String value, ExtractionFn fn)
|
|
||||||
{
|
|
||||||
this.dimension = dimension;
|
|
||||||
this.value = Strings.nullToEmpty(value);
|
|
||||||
this.fn = fn;
|
|
||||||
}
|
|
||||||
|
|
||||||
private List<Filter> makeFilters(BitmapIndexSelector selector)
|
|
||||||
{
|
|
||||||
Indexed<String> allDimVals = selector.getDimensionValues(dimension);
|
|
||||||
final List<Filter> filters = Lists.newArrayList();
|
|
||||||
if (allDimVals == null) {
|
|
||||||
allDimVals = new Indexed<String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Iterator<String> iterator()
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Class<? extends String> getClazz()
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int size() { return 1; }
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String get(int index) { return null;}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int indexOf(String value)
|
|
||||||
{
|
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
for (int i = 0; i < allDimVals.size(); i++) {
|
|
||||||
String dimVal = allDimVals.get(i);
|
|
||||||
if (value.equals(Strings.nullToEmpty(fn.apply(dimVal)))) {
|
|
||||||
filters.add(new SelectorFilter(dimension, dimVal));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return filters;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
|
|
||||||
{
|
|
||||||
final List<Filter> filters = makeFilters(selector);
|
|
||||||
if (filters.isEmpty()) {
|
|
||||||
return selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
|
||||||
}
|
|
||||||
return new OrFilter(makeFilters(selector)).getBitmapIndex(selector);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
|
||||||
{
|
|
||||||
return factory.makeValueMatcher(
|
|
||||||
dimension, new Predicate<String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public boolean apply(String input)
|
|
||||||
{
|
|
||||||
// Assuming that a null/absent/empty dimension are equivalent from the druid perspective
|
|
||||||
return value.equals(Strings.nullToEmpty(fn.apply(Strings.emptyToNull(input))));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -23,14 +23,18 @@ import com.google.common.base.Function;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
import com.metamx.collections.bitmap.BitmapFactory;
|
import com.metamx.collections.bitmap.BitmapFactory;
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||||
|
import com.metamx.collections.bitmap.MutableBitmap;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.filter.BitmapIndexSelector;
|
import io.druid.query.filter.BitmapIndexSelector;
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
import io.druid.query.filter.ValueMatcher;
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.query.filter.ValueMatcherFactory;
|
import io.druid.query.filter.ValueMatcherFactory;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -39,28 +43,46 @@ public class InFilter implements Filter
|
||||||
{
|
{
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
private final Set<String> values;
|
private final Set<String> values;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
public InFilter(String dimension, Set<String> values)
|
public InFilter(String dimension, Set<String> values, ExtractionFn extractionFn)
|
||||||
{
|
{
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.values = values;
|
this.values = values;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
|
public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
return selector.getBitmapFactory().union(
|
if (extractionFn == null) {
|
||||||
Iterables.transform(
|
return selector.getBitmapFactory().union(
|
||||||
values, new Function<String, ImmutableBitmap>()
|
Iterables.transform(
|
||||||
{
|
values, new Function<String, ImmutableBitmap>()
|
||||||
@Override
|
|
||||||
public ImmutableBitmap apply(String value)
|
|
||||||
{
|
{
|
||||||
return selector.getBitmapIndex(dimension, value);
|
@Override
|
||||||
|
public ImmutableBitmap apply(String value)
|
||||||
|
{
|
||||||
|
return selector.getBitmapIndex(dimension, value);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
)
|
||||||
)
|
);
|
||||||
);
|
} else {
|
||||||
|
Iterable<String> allDimVals = selector.getDimensionValues(dimension);
|
||||||
|
if (allDimVals == null) {
|
||||||
|
allDimVals = Lists.newArrayList((String) null);
|
||||||
|
}
|
||||||
|
|
||||||
|
List<ImmutableBitmap> bitmaps = Lists.newArrayList();
|
||||||
|
for (String dimVal : allDimVals) {
|
||||||
|
System.out.println(dimVal);
|
||||||
|
if (values.contains(Strings.nullToEmpty(extractionFn.apply(dimVal)))) {
|
||||||
|
bitmaps.add(selector.getBitmapIndex(dimension, dimVal));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return selector.getBitmapFactory().union(bitmaps);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -72,6 +94,9 @@ public class InFilter implements Filter
|
||||||
@Override
|
@Override
|
||||||
public boolean apply(String input)
|
public boolean apply(String input)
|
||||||
{
|
{
|
||||||
|
if (extractionFn != null) {
|
||||||
|
input = extractionFn.apply(input);
|
||||||
|
}
|
||||||
return values.contains(Strings.nullToEmpty(input));
|
return values.contains(Strings.nullToEmpty(input));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||||
import com.metamx.common.guava.FunctionalIterable;
|
import com.metamx.common.guava.FunctionalIterable;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.filter.BitmapIndexSelector;
|
import io.druid.query.filter.BitmapIndexSelector;
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
import io.druid.query.filter.ValueMatcher;
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
@ -38,11 +39,13 @@ public class JavaScriptFilter implements Filter
|
||||||
{
|
{
|
||||||
private final JavaScriptPredicate predicate;
|
private final JavaScriptPredicate predicate;
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
public JavaScriptFilter(String dimension, final String script)
|
public JavaScriptFilter(String dimension, final String script, ExtractionFn extractionFn)
|
||||||
{
|
{
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.predicate = new JavaScriptPredicate(script);
|
this.predicate = new JavaScriptPredicate(script, extractionFn);
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -52,8 +55,11 @@ public class JavaScriptFilter implements Filter
|
||||||
try {
|
try {
|
||||||
final Indexed<String> dimValues = selector.getDimensionValues(dimension);
|
final Indexed<String> dimValues = selector.getDimensionValues(dimension);
|
||||||
ImmutableBitmap bitmap;
|
ImmutableBitmap bitmap;
|
||||||
if (dimValues == null) {
|
if (dimValues == null || dimValues.size() == 0) {
|
||||||
bitmap = selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
bitmap = selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
||||||
|
if (predicate.applyInContext(cx, null)) {
|
||||||
|
bitmap = selector.getBitmapFactory().complement(bitmap, selector.getNumRows());
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
bitmap = selector.getBitmapFactory().union(
|
bitmap = selector.getBitmapFactory().union(
|
||||||
FunctionalIterable.create(dimValues)
|
FunctionalIterable.create(dimValues)
|
||||||
|
@ -98,11 +104,13 @@ public class JavaScriptFilter implements Filter
|
||||||
final ScriptableObject scope;
|
final ScriptableObject scope;
|
||||||
final Function fnApply;
|
final Function fnApply;
|
||||||
final String script;
|
final String script;
|
||||||
|
final ExtractionFn extractionFn;
|
||||||
|
|
||||||
public JavaScriptPredicate(final String script)
|
public JavaScriptPredicate(final String script, final ExtractionFn extractionFn)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(script, "script must not be null");
|
Preconditions.checkNotNull(script, "script must not be null");
|
||||||
this.script = script;
|
this.script = script;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
|
|
||||||
final Context cx = Context.enter();
|
final Context cx = Context.enter();
|
||||||
try {
|
try {
|
||||||
|
@ -127,11 +135,13 @@ public class JavaScriptFilter implements Filter
|
||||||
finally {
|
finally {
|
||||||
Context.exit();
|
Context.exit();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean applyInContext(Context cx, String input)
|
public boolean applyInContext(Context cx, String input)
|
||||||
{
|
{
|
||||||
|
if (extractionFn != null) {
|
||||||
|
input = extractionFn.apply(input);
|
||||||
|
}
|
||||||
return Context.toBoolean(fnApply.call(cx, scope, scope, new String[]{input}));
|
return Context.toBoolean(fnApply.call(cx, scope, scope, new String[]{input}));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package io.druid.segment.filter;
|
package io.druid.segment.filter;
|
||||||
|
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
@ -29,7 +30,8 @@ public class RegexFilter extends DimensionPredicateFilter
|
||||||
{
|
{
|
||||||
public RegexFilter(
|
public RegexFilter(
|
||||||
String dimension,
|
String dimension,
|
||||||
final String pattern
|
final String pattern,
|
||||||
|
final ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(
|
super(
|
||||||
|
@ -43,7 +45,8 @@ public class RegexFilter extends DimensionPredicateFilter
|
||||||
{
|
{
|
||||||
return (input != null) && compiled.matcher(input).find();
|
return (input != null) && compiled.matcher(input).find();
|
||||||
}
|
}
|
||||||
}
|
},
|
||||||
|
extractionFn
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.segment.filter;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.search.search.SearchQuerySpec;
|
import io.druid.query.search.search.SearchQuerySpec;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -33,7 +34,8 @@ public class SearchQueryFilter extends DimensionPredicateFilter
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public SearchQueryFilter(
|
public SearchQueryFilter(
|
||||||
@JsonProperty("dimension") String dimension,
|
@JsonProperty("dimension") String dimension,
|
||||||
@JsonProperty("query") final SearchQuerySpec query
|
@JsonProperty("query") final SearchQuerySpec query,
|
||||||
|
@JsonProperty("extractionFn") final ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(
|
super(
|
||||||
|
@ -45,7 +47,8 @@ public class SearchQueryFilter extends DimensionPredicateFilter
|
||||||
{
|
{
|
||||||
return query.accept(input);
|
return query.accept(input);
|
||||||
}
|
}
|
||||||
}
|
},
|
||||||
|
extractionFn
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,11 +19,20 @@
|
||||||
|
|
||||||
package io.druid.segment.filter;
|
package io.druid.segment.filter;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import com.google.common.base.Strings;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.filter.BitmapIndexSelector;
|
import io.druid.query.filter.BitmapIndexSelector;
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
import io.druid.query.filter.ValueMatcher;
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.query.filter.ValueMatcherFactory;
|
import io.druid.query.filter.ValueMatcherFactory;
|
||||||
|
import io.druid.segment.data.Indexed;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -31,25 +40,68 @@ public class SelectorFilter implements Filter
|
||||||
{
|
{
|
||||||
private final String dimension;
|
private final String dimension;
|
||||||
private final String value;
|
private final String value;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
|
||||||
public SelectorFilter(
|
public SelectorFilter(
|
||||||
String dimension,
|
String dimension,
|
||||||
String value
|
String value,
|
||||||
|
ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.value = value;
|
this.value = Strings.nullToEmpty(value);
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
|
public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
return selector.getBitmapIndex(dimension, value);
|
if (extractionFn == null) {
|
||||||
|
return selector.getBitmapIndex(dimension, value);
|
||||||
|
} else {
|
||||||
|
final List<Filter> filters = makeFiltersUsingExtractionFn(selector);
|
||||||
|
if (filters.isEmpty()) {
|
||||||
|
return selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
||||||
|
}
|
||||||
|
return new OrFilter(filters).getBitmapIndex(selector);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
||||||
{
|
{
|
||||||
return factory.makeValueMatcher(dimension, value);
|
if (extractionFn == null) {
|
||||||
|
return factory.makeValueMatcher(dimension, value);
|
||||||
|
} else {
|
||||||
|
return factory.makeValueMatcher(
|
||||||
|
dimension, new Predicate<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean apply(String input)
|
||||||
|
{
|
||||||
|
// Assuming that a null/absent/empty dimension are equivalent from the druid perspective
|
||||||
|
return value.equals(Strings.nullToEmpty(extractionFn.apply(input)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<Filter> makeFiltersUsingExtractionFn(BitmapIndexSelector selector)
|
||||||
|
{
|
||||||
|
final List<Filter> filters = Lists.newArrayList();
|
||||||
|
|
||||||
|
Iterable<String> allDimVals = selector.getDimensionValues(dimension);
|
||||||
|
if (allDimVals == null) {
|
||||||
|
allDimVals = Lists.newArrayList((String) null);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (String dimVal : allDimVals) {
|
||||||
|
if (value.equals(Strings.nullToEmpty(extractionFn.apply(dimVal)))) {
|
||||||
|
filters.add(new SelectorFilter(dimension, dimVal, null));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return filters;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -196,7 +196,7 @@ public class MultiValuedDimensionTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.setDimFilter(
|
.setDimFilter(
|
||||||
new SelectorDimFilter("tags", "t3")
|
new SelectorDimFilter("tags", "t3", null)
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -244,7 +244,7 @@ public class MultiValuedDimensionTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.setDimFilter(
|
.setDimFilter(
|
||||||
new SelectorDimFilter("tags", "t3")
|
new SelectorDimFilter("tags", "t3", null)
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -284,7 +284,7 @@ public class MultiValuedDimensionTest
|
||||||
}
|
}
|
||||||
))
|
))
|
||||||
.threshold(5)
|
.threshold(5)
|
||||||
.filters(new SelectorDimFilter("tags", "t3")).build();
|
.filters(new SelectorDimFilter("tags", "t3", null)).build();
|
||||||
|
|
||||||
QueryRunnerFactory factory = new TopNQueryRunnerFactory(
|
QueryRunnerFactory factory = new TopNQueryRunnerFactory(
|
||||||
TestQueryRunners.getPool(),
|
TestQueryRunners.getPool(),
|
||||||
|
|
|
@ -22,11 +22,18 @@ package io.druid.query.aggregation;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||||
import io.druid.query.filter.AndDimFilter;
|
import io.druid.query.filter.AndDimFilter;
|
||||||
|
import io.druid.query.filter.BoundDimFilter;
|
||||||
import io.druid.query.filter.DimFilter;
|
import io.druid.query.filter.DimFilter;
|
||||||
|
import io.druid.query.filter.InDimFilter;
|
||||||
|
import io.druid.query.filter.JavaScriptDimFilter;
|
||||||
import io.druid.query.filter.NotDimFilter;
|
import io.druid.query.filter.NotDimFilter;
|
||||||
import io.druid.query.filter.OrDimFilter;
|
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.filter.SelectorDimFilter;
|
||||||
|
import io.druid.query.search.search.ContainsSearchQuerySpec;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
|
@ -37,6 +44,8 @@ import io.druid.segment.data.IndexedInts;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
public class FilteredAggregatorTest
|
public class FilteredAggregatorTest
|
||||||
{
|
{
|
||||||
private void aggregate(TestFloatColumnSelector selector, FilteredAggregator agg)
|
private void aggregate(TestFloatColumnSelector selector, FilteredAggregator agg)
|
||||||
|
@ -53,7 +62,7 @@ public class FilteredAggregatorTest
|
||||||
|
|
||||||
FilteredAggregatorFactory factory = new FilteredAggregatorFactory(
|
FilteredAggregatorFactory factory = new FilteredAggregatorFactory(
|
||||||
new DoubleSumAggregatorFactory("billy", "value"),
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
new SelectorDimFilter("dim", "a")
|
new SelectorDimFilter("dim", "a", null)
|
||||||
);
|
);
|
||||||
|
|
||||||
FilteredAggregator agg = (FilteredAggregator) factory.factorize(
|
FilteredAggregator agg = (FilteredAggregator) factory.factorize(
|
||||||
|
@ -175,19 +184,10 @@ public class FilteredAggregatorTest
|
||||||
|
|
||||||
FilteredAggregatorFactory factory = new FilteredAggregatorFactory(
|
FilteredAggregatorFactory factory = new FilteredAggregatorFactory(
|
||||||
new DoubleSumAggregatorFactory("billy", "value"),
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
new NotDimFilter(new SelectorDimFilter("dim", "b"))
|
new NotDimFilter(new SelectorDimFilter("dim", "b", null))
|
||||||
);
|
);
|
||||||
|
|
||||||
FilteredAggregator agg = (FilteredAggregator) factory.factorize(
|
validateFilteredAggs(factory, values, selector);
|
||||||
makeColumnSelector(selector)
|
|
||||||
);
|
|
||||||
|
|
||||||
Assert.assertEquals("billy", agg.getName());
|
|
||||||
|
|
||||||
double expectedFirst = new Float(values[0]).doubleValue();
|
|
||||||
double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst;
|
|
||||||
double expectedThird = expectedSecond;
|
|
||||||
assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -198,7 +198,7 @@ public class FilteredAggregatorTest
|
||||||
|
|
||||||
FilteredAggregatorFactory factory = new FilteredAggregatorFactory(
|
FilteredAggregatorFactory factory = new FilteredAggregatorFactory(
|
||||||
new DoubleSumAggregatorFactory("billy", "value"),
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
new OrDimFilter(Lists.<DimFilter>newArrayList(new SelectorDimFilter("dim", "a"), new SelectorDimFilter("dim", "b")))
|
new OrDimFilter(Lists.<DimFilter>newArrayList(new SelectorDimFilter("dim", "a", null), new SelectorDimFilter("dim", "b", null)))
|
||||||
);
|
);
|
||||||
|
|
||||||
FilteredAggregator agg = (FilteredAggregator) factory.factorize(
|
FilteredAggregator agg = (FilteredAggregator) factory.factorize(
|
||||||
|
@ -221,8 +221,108 @@ public class FilteredAggregatorTest
|
||||||
|
|
||||||
FilteredAggregatorFactory factory = new FilteredAggregatorFactory(
|
FilteredAggregatorFactory factory = new FilteredAggregatorFactory(
|
||||||
new DoubleSumAggregatorFactory("billy", "value"),
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
new AndDimFilter(Lists.<DimFilter>newArrayList(new NotDimFilter(new SelectorDimFilter("dim", "b")), new SelectorDimFilter("dim", "a"))));
|
new AndDimFilter(Lists.<DimFilter>newArrayList(new NotDimFilter(new SelectorDimFilter("dim", "b", null)), new SelectorDimFilter("dim", "a", null))));
|
||||||
|
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAggregateWithPredicateFilters()
|
||||||
|
{
|
||||||
|
final float[] values = {0.15f, 0.27f};
|
||||||
|
TestFloatColumnSelector selector;
|
||||||
|
FilteredAggregatorFactory factory;
|
||||||
|
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new BoundDimFilter("dim", "a", "a", false, false, true, null)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new RegexDimFilter("dim", "a", null)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new SearchQueryDimFilter("dim", new ContainsSearchQuerySpec("a", true), null)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
|
||||||
|
String jsFn = "function(x) { return(x === 'a') }";
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new JavaScriptDimFilter("dim", jsFn, null)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAggregateWithExtractionFns()
|
||||||
|
{
|
||||||
|
final float[] values = {0.15f, 0.27f};
|
||||||
|
TestFloatColumnSelector selector;
|
||||||
|
FilteredAggregatorFactory factory;
|
||||||
|
|
||||||
|
String extractionJsFn = "function(str) { return str + 'AARDVARK'; }";
|
||||||
|
ExtractionFn extractionFn = new JavaScriptExtractionFn(extractionJsFn, false);
|
||||||
|
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new SelectorDimFilter("dim", "aAARDVARK", extractionFn)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new InDimFilter("dim", Arrays.asList("NOT-aAARDVARK", "FOOBAR", "aAARDVARK"), extractionFn)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new BoundDimFilter("dim", "aAARDVARK", "aAARDVARK", false, false, true, extractionFn)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new RegexDimFilter("dim", "aAARDVARK", extractionFn)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new SearchQueryDimFilter("dim", new ContainsSearchQuerySpec("aAARDVARK", true), extractionFn)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
|
||||||
|
String jsFn = "function(x) { return(x === 'aAARDVARK') }";
|
||||||
|
factory = new FilteredAggregatorFactory(
|
||||||
|
new DoubleSumAggregatorFactory("billy", "value"),
|
||||||
|
new JavaScriptDimFilter("dim", jsFn, extractionFn)
|
||||||
|
);
|
||||||
|
selector = new TestFloatColumnSelector(values);
|
||||||
|
validateFilteredAggs(factory, values, selector);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void validateFilteredAggs(
|
||||||
|
FilteredAggregatorFactory factory,
|
||||||
|
float[] values,
|
||||||
|
TestFloatColumnSelector selector
|
||||||
|
)
|
||||||
|
{
|
||||||
FilteredAggregator agg = (FilteredAggregator) factory.factorize(
|
FilteredAggregator agg = (FilteredAggregator) factory.factorize(
|
||||||
makeColumnSelector(selector)
|
makeColumnSelector(selector)
|
||||||
);
|
);
|
||||||
|
@ -232,7 +332,7 @@ public class FilteredAggregatorTest
|
||||||
double expectedFirst = new Float(values[0]).doubleValue();
|
double expectedFirst = new Float(values[0]).doubleValue();
|
||||||
double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst;
|
double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst;
|
||||||
double expectedThird = expectedSecond;
|
double expectedThird = expectedSecond;
|
||||||
|
|
||||||
assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird);
|
assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,8 @@ import com.google.inject.Injector;
|
||||||
import com.google.inject.Key;
|
import com.google.inject.Key;
|
||||||
import io.druid.guice.GuiceInjectors;
|
import io.druid.guice.GuiceInjectors;
|
||||||
import io.druid.guice.annotations.Json;
|
import io.druid.guice.annotations.Json;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
|
@ -42,17 +44,22 @@ public class BoundDimFilterTest
|
||||||
|
|
||||||
private final BoundDimFilter boundDimFilter;
|
private final BoundDimFilter boundDimFilter;
|
||||||
|
|
||||||
|
private static final ExtractionFn extractionFn = new RegexDimExtractionFn(".*", false, null);
|
||||||
|
|
||||||
@Parameterized.Parameters
|
@Parameterized.Parameters
|
||||||
public static Iterable<Object[]> constructorFeeder(){
|
public static Iterable<Object[]> constructorFeeder(){
|
||||||
|
|
||||||
return ImmutableList.of(new Object[]{new BoundDimFilter("dimension", "12", "15", null, null, null)},
|
return ImmutableList.of(
|
||||||
new Object[]{new BoundDimFilter("dimension", "12", "15", null, true, false)},
|
new Object[]{new BoundDimFilter("dimension", "12", "15", null, null, null, null)},
|
||||||
new Object[]{new BoundDimFilter("dimension", "12", "15", null, null, true)},
|
new Object[]{new BoundDimFilter("dimension", "12", "15", null, true, false, null)},
|
||||||
new Object[]{new BoundDimFilter("dimension", null, "15", null, true, true)},
|
new Object[]{new BoundDimFilter("dimension", "12", "15", null, null, true, null)},
|
||||||
new Object[]{new BoundDimFilter("dimension", "12", "15", true, null, null)},
|
new Object[]{new BoundDimFilter("dimension", null, "15", null, true, true, null)},
|
||||||
new Object[]{new BoundDimFilter("dimension", "12", null, true, null, true)},
|
new Object[]{new BoundDimFilter("dimension", "12", "15", true, null, null, null)},
|
||||||
new Object[]{new BoundDimFilter("dimension", "12", "15", true, true, true)},
|
new Object[]{new BoundDimFilter("dimension", "12", null, true, null, true, null)},
|
||||||
new Object[]{new BoundDimFilter("dimension", "12", "15", true, true, false)});
|
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)}
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -68,10 +75,24 @@ public class BoundDimFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testGetCacheKey()
|
public void testGetCacheKey()
|
||||||
{
|
{
|
||||||
BoundDimFilter boundDimFilter = new BoundDimFilter("dimension", "12", "15", null, null, true);
|
BoundDimFilter boundDimFilter = new BoundDimFilter("dimension", "12", "15", null, null, true, null);
|
||||||
BoundDimFilter boundDimFilterCopy = new BoundDimFilter("dimension", "12", "15", false, false, true);
|
BoundDimFilter boundDimFilterCopy = new BoundDimFilter("dimension", "12", "15", false, false, true, null);
|
||||||
Assert.assertArrayEquals(boundDimFilter.getCacheKey(), boundDimFilterCopy.getCacheKey());
|
Assert.assertArrayEquals(boundDimFilter.getCacheKey(), boundDimFilterCopy.getCacheKey());
|
||||||
BoundDimFilter anotherBoundDimFilter = new BoundDimFilter("dimension", "12", "15", true, null, false);
|
BoundDimFilter anotherBoundDimFilter = new BoundDimFilter("dimension", "12", "15", true, null, false, null);
|
||||||
Assert.assertFalse(Arrays.equals(anotherBoundDimFilter.getCacheKey(), boundDimFilter.getCacheKey()));
|
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);
|
||||||
|
Assert.assertFalse(Arrays.equals(boundDimFilter.getCacheKey(), boundDimFilterWithExtract.getCacheKey()));
|
||||||
|
Assert.assertArrayEquals(boundDimFilterWithExtract.getCacheKey(), boundDimFilterWithExtractCopy.getCacheKey());
|
||||||
|
}
|
||||||
|
|
||||||
|
@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);
|
||||||
|
|
||||||
|
Assert.assertNotEquals(boundDimFilter.hashCode(), boundDimFilterWithExtract.hashCode());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.google.inject.Injector;
|
||||||
import com.google.inject.Key;
|
import com.google.inject.Key;
|
||||||
import io.druid.guice.GuiceInjectors;
|
import io.druid.guice.GuiceInjectors;
|
||||||
import io.druid.guice.annotations.Json;
|
import io.druid.guice.annotations.Json;
|
||||||
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -35,7 +36,7 @@ public class InDimFilterSerDesrTest
|
||||||
{
|
{
|
||||||
private static ObjectMapper mapper;
|
private static ObjectMapper mapper;
|
||||||
|
|
||||||
private final String actualInFilter = "{\"type\":\"in\",\"dimension\":\"dimTest\",\"values\":[\"good\",\"bad\"]}";
|
private final String actualInFilter = "{\"type\":\"in\",\"dimension\":\"dimTest\",\"values\":[\"good\",\"bad\"],\"extractionFn\":null}";
|
||||||
@Before
|
@Before
|
||||||
public void setUp()
|
public void setUp()
|
||||||
{
|
{
|
||||||
|
@ -47,14 +48,14 @@ public class InDimFilterSerDesrTest
|
||||||
public void testDeserialization() throws IOException
|
public void testDeserialization() throws IOException
|
||||||
{
|
{
|
||||||
final InDimFilter actualInDimFilter = mapper.reader(DimFilter.class).readValue(actualInFilter);
|
final InDimFilter actualInDimFilter = mapper.reader(DimFilter.class).readValue(actualInFilter);
|
||||||
final InDimFilter expectedInDimFilter = new InDimFilter("dimTest", Arrays.asList("good", "bad"));
|
final InDimFilter expectedInDimFilter = new InDimFilter("dimTest", Arrays.asList("good", "bad"), null);
|
||||||
Assert.assertEquals(expectedInDimFilter, actualInDimFilter);
|
Assert.assertEquals(expectedInDimFilter, actualInDimFilter);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSerialization() throws IOException
|
public void testSerialization() throws IOException
|
||||||
{
|
{
|
||||||
final InDimFilter dimInFilter = new InDimFilter("dimTest", Arrays.asList("good", "bad"));
|
final InDimFilter dimInFilter = new InDimFilter("dimTest", Arrays.asList("good", "bad"), null);
|
||||||
final String expectedInFilter = mapper.writeValueAsString(dimInFilter);
|
final String expectedInFilter = mapper.writeValueAsString(dimInFilter);
|
||||||
Assert.assertEquals(expectedInFilter, actualInFilter);
|
Assert.assertEquals(expectedInFilter, actualInFilter);
|
||||||
}
|
}
|
||||||
|
@ -62,9 +63,14 @@ public class InDimFilterSerDesrTest
|
||||||
@Test
|
@Test
|
||||||
public void testGetCacheKey()
|
public void testGetCacheKey()
|
||||||
{
|
{
|
||||||
final InDimFilter inDimFilter_1 = new InDimFilter("dimTest", Arrays.asList("good", "bad"));
|
final InDimFilter inDimFilter_1 = new InDimFilter("dimTest", Arrays.asList("good", "bad"), null);
|
||||||
final InDimFilter inDimFilter_2 = new InDimFilter("dimTest", Arrays.asList("good,bad"));
|
final InDimFilter inDimFilter_2 = new InDimFilter("dimTest", Arrays.asList("good,bad"), null);
|
||||||
Assert.assertNotEquals(inDimFilter_1.getCacheKey(), inDimFilter_2.getCacheKey());
|
Assert.assertNotEquals(inDimFilter_1.getCacheKey(), inDimFilter_2.getCacheKey());
|
||||||
|
|
||||||
|
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
|
||||||
|
final InDimFilter inDimFilter_3 = new InDimFilter("dimTest", Arrays.asList("good", "bad"), regexFn);
|
||||||
|
final InDimFilter inDimFilter_4 = new InDimFilter("dimTest", Arrays.asList("good,bad"), regexFn);
|
||||||
|
Assert.assertNotEquals(inDimFilter_3.getCacheKey(), inDimFilter_4.getCacheKey());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.query.filter;
|
package io.druid.query.filter;
|
||||||
|
|
||||||
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -30,8 +31,12 @@ public class JavaScriptDimFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testGetCacheKey()
|
public void testGetCacheKey()
|
||||||
{
|
{
|
||||||
JavaScriptDimFilter javaScriptDimFilter = new JavaScriptDimFilter("dim", "fn");
|
JavaScriptDimFilter javaScriptDimFilter = new JavaScriptDimFilter("dim", "fn", null);
|
||||||
JavaScriptDimFilter javaScriptDimFilter2 = new JavaScriptDimFilter("di", "mfn");
|
JavaScriptDimFilter javaScriptDimFilter2 = new JavaScriptDimFilter("di", "mfn", null);
|
||||||
Assert.assertFalse(Arrays.equals(javaScriptDimFilter.getCacheKey(), javaScriptDimFilter2.getCacheKey()));
|
Assert.assertFalse(Arrays.equals(javaScriptDimFilter.getCacheKey(), javaScriptDimFilter2.getCacheKey()));
|
||||||
|
|
||||||
|
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
|
||||||
|
JavaScriptDimFilter javaScriptDimFilter3 = new JavaScriptDimFilter("dim", "fn", regexFn);
|
||||||
|
Assert.assertFalse(Arrays.equals(javaScriptDimFilter.getCacheKey(), javaScriptDimFilter3.getCacheKey()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.query.filter;
|
package io.druid.query.filter;
|
||||||
|
|
||||||
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -30,8 +31,13 @@ public class RegexDimFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testGetCacheKey()
|
public void testGetCacheKey()
|
||||||
{
|
{
|
||||||
RegexDimFilter regexDimFilter = new RegexDimFilter("dim", "reg");
|
RegexDimFilter regexDimFilter = new RegexDimFilter("dim", "reg", null);
|
||||||
RegexDimFilter regexDimFilter2 = new RegexDimFilter("di", "mreg");
|
RegexDimFilter regexDimFilter2 = new RegexDimFilter("di", "mreg", null);
|
||||||
Assert.assertFalse(Arrays.equals(regexDimFilter.getCacheKey(), regexDimFilter2.getCacheKey()));
|
Assert.assertFalse(Arrays.equals(regexDimFilter.getCacheKey(), regexDimFilter2.getCacheKey()));
|
||||||
|
|
||||||
|
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
|
||||||
|
RegexDimFilter regexDimFilter3 = new RegexDimFilter("dim", "reg", regexFn);
|
||||||
|
Assert.assertFalse(Arrays.equals(regexDimFilter.getCacheKey(), regexDimFilter3.getCacheKey()));
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package io.druid.query.filter;
|
package io.druid.query.filter;
|
||||||
|
|
||||||
import com.metamx.common.StringUtils;
|
import com.metamx.common.StringUtils;
|
||||||
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
import io.druid.query.search.search.SearchQuerySpec;
|
import io.druid.query.search.search.SearchQuerySpec;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -32,35 +33,64 @@ public class SearchQueryDimFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testGetCacheKey()
|
public void testGetCacheKey()
|
||||||
{
|
{
|
||||||
SearchQueryDimFilter searchQueryDimFilter = new SearchQueryDimFilter("dim", new SearchQuerySpec()
|
SearchQueryDimFilter searchQueryDimFilter = new SearchQueryDimFilter(
|
||||||
{
|
"dim",
|
||||||
@Override
|
new SearchQuerySpec()
|
||||||
public boolean accept(String dimVal)
|
{
|
||||||
{
|
@Override
|
||||||
return false;
|
public boolean accept(String dimVal)
|
||||||
}
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] getCacheKey()
|
public byte[] getCacheKey()
|
||||||
{
|
{
|
||||||
return StringUtils.toUtf8("value");
|
return StringUtils.toUtf8("value");
|
||||||
}
|
}
|
||||||
});
|
},
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
SearchQueryDimFilter searchQueryDimFilter2 = new SearchQueryDimFilter("di", new SearchQuerySpec()
|
SearchQueryDimFilter searchQueryDimFilter2 = new SearchQueryDimFilter(
|
||||||
{
|
"di",
|
||||||
@Override
|
new SearchQuerySpec()
|
||||||
public boolean accept(String dimVal)
|
{
|
||||||
{
|
@Override
|
||||||
return false;
|
public boolean accept(String dimVal)
|
||||||
}
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] getCacheKey()
|
public byte[] getCacheKey()
|
||||||
{
|
{
|
||||||
return StringUtils.toUtf8("mvalue");
|
return StringUtils.toUtf8("mvalue");
|
||||||
}
|
}
|
||||||
});
|
},
|
||||||
|
null
|
||||||
|
);
|
||||||
Assert.assertFalse(Arrays.equals(searchQueryDimFilter.getCacheKey(), searchQueryDimFilter2.getCacheKey()));
|
Assert.assertFalse(Arrays.equals(searchQueryDimFilter.getCacheKey(), searchQueryDimFilter2.getCacheKey()));
|
||||||
|
|
||||||
|
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
|
||||||
|
SearchQueryDimFilter searchQueryDimFilter3 = new SearchQueryDimFilter(
|
||||||
|
"dim",
|
||||||
|
new SearchQuerySpec()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean accept(String dimVal)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return StringUtils.toUtf8("value");
|
||||||
|
}
|
||||||
|
},
|
||||||
|
regexFn
|
||||||
|
);
|
||||||
|
Assert.assertFalse(Arrays.equals(searchQueryDimFilter.getCacheKey(), searchQueryDimFilter3.getCacheKey()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.query.filter;
|
package io.druid.query.filter;
|
||||||
|
|
||||||
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -29,15 +30,40 @@ public class SelectorDimFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testGetCacheKey()
|
public void testGetCacheKey()
|
||||||
{
|
{
|
||||||
SelectorDimFilter selectorDimFilter = new SelectorDimFilter("abc", "d");
|
SelectorDimFilter selectorDimFilter = new SelectorDimFilter("abc", "d", null);
|
||||||
SelectorDimFilter selectorDimFilter2 = new SelectorDimFilter("ab", "cd");
|
SelectorDimFilter selectorDimFilter2 = new SelectorDimFilter("ab", "cd", null);
|
||||||
Assert.assertFalse(Arrays.equals(selectorDimFilter.getCacheKey(), selectorDimFilter2.getCacheKey()));
|
Assert.assertFalse(Arrays.equals(selectorDimFilter.getCacheKey(), selectorDimFilter2.getCacheKey()));
|
||||||
|
|
||||||
|
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
|
||||||
|
SelectorDimFilter selectorDimFilter3 = new SelectorDimFilter("abc", "d", regexFn);
|
||||||
|
Assert.assertFalse(Arrays.equals(selectorDimFilter.getCacheKey(), selectorDimFilter3.getCacheKey()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testToString()
|
||||||
|
{
|
||||||
|
SelectorDimFilter selectorDimFilter = new SelectorDimFilter("abc", "d", null);
|
||||||
|
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
|
||||||
|
SelectorDimFilter selectorDimFilter2 = new SelectorDimFilter("abc", "d", regexFn);
|
||||||
|
|
||||||
|
Assert.assertEquals("abc = d", selectorDimFilter.toString());
|
||||||
|
Assert.assertEquals("regex(.*)(abc) = d", selectorDimFilter2.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testHashCode()
|
||||||
|
{
|
||||||
|
SelectorDimFilter selectorDimFilter = new SelectorDimFilter("abc", "d", null);
|
||||||
|
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
|
||||||
|
SelectorDimFilter selectorDimFilter2 = new SelectorDimFilter("abc", "d", regexFn);
|
||||||
|
|
||||||
|
Assert.assertNotEquals(selectorDimFilter.hashCode(), selectorDimFilter2.hashCode());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSimpleOptimize()
|
public void testSimpleOptimize()
|
||||||
{
|
{
|
||||||
SelectorDimFilter selectorDimFilter = new SelectorDimFilter("abc", "d");
|
SelectorDimFilter selectorDimFilter = new SelectorDimFilter("abc", "d", null);
|
||||||
DimFilter filter = new AndDimFilter(
|
DimFilter filter = new AndDimFilter(
|
||||||
Arrays.<DimFilter>asList(
|
Arrays.<DimFilter>asList(
|
||||||
new OrDimFilter(
|
new OrDimFilter(
|
||||||
|
|
|
@ -64,6 +64,11 @@ import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.dimension.ExtractionDimensionSpec;
|
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||||
|
import io.druid.query.filter.AndDimFilter;
|
||||||
|
import io.druid.query.filter.BoundDimFilter;
|
||||||
|
import io.druid.query.filter.InDimFilter;
|
||||||
|
import io.druid.query.filter.SearchQueryDimFilter;
|
||||||
import io.druid.query.lookup.LookupExtractionFn;
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
import io.druid.query.extraction.MapLookupExtractor;
|
import io.druid.query.extraction.MapLookupExtractor;
|
||||||
import io.druid.query.extraction.RegexDimExtractionFn;
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
|
@ -82,6 +87,7 @@ import io.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||||
import io.druid.query.groupby.orderby.LimitSpec;
|
import io.druid.query.groupby.orderby.LimitSpec;
|
||||||
import io.druid.query.groupby.orderby.OrderByColumnSpec;
|
import io.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||||
import io.druid.query.ordering.StringComparators;
|
import io.druid.query.ordering.StringComparators;
|
||||||
|
import io.druid.query.search.search.ContainsSearchQuerySpec;
|
||||||
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||||
import io.druid.segment.TestHelper;
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
|
@ -101,6 +107,7 @@ import org.junit.runners.Parameterized;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
@ -2506,7 +2513,7 @@ public class GroupByQueryRunnerTest
|
||||||
.builder()
|
.builder()
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.setInterval("2011-04-02/2011-04-04")
|
.setInterval("2011-04-02/2011-04-04")
|
||||||
.setDimFilter(new RegexDimFilter("quality", "auto.*"))
|
.setDimFilter(new RegexDimFilter("quality", "auto.*", null))
|
||||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "quality")))
|
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "quality")))
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.<AggregatorFactory>asList(
|
Arrays.<AggregatorFactory>asList(
|
||||||
|
@ -2715,7 +2722,7 @@ public class GroupByQueryRunnerTest
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||||
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }"))
|
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }", null))
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
QueryRunnerTestHelper.rowsCount,
|
QueryRunnerTestHelper.rowsCount,
|
||||||
|
@ -2774,7 +2781,7 @@ public class GroupByQueryRunnerTest
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||||
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }"))
|
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }", null))
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
QueryRunnerTestHelper.rowsCount,
|
QueryRunnerTestHelper.rowsCount,
|
||||||
|
@ -2842,7 +2849,7 @@ public class GroupByQueryRunnerTest
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||||
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }"))
|
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }", null))
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
QueryRunnerTestHelper.rowsCount,
|
QueryRunnerTestHelper.rowsCount,
|
||||||
|
@ -3012,15 +3019,15 @@ public class GroupByQueryRunnerTest
|
||||||
.setDimFilter(
|
.setDimFilter(
|
||||||
new OrDimFilter(
|
new OrDimFilter(
|
||||||
Lists.<DimFilter>newArrayList(
|
Lists.<DimFilter>newArrayList(
|
||||||
new SelectorDimFilter("quality", "automotive"),
|
new SelectorDimFilter("quality", "automotive", null),
|
||||||
new SelectorDimFilter("quality", "premium"),
|
new SelectorDimFilter("quality", "premium", null),
|
||||||
new SelectorDimFilter("quality", "mezzanine"),
|
new SelectorDimFilter("quality", "mezzanine", null),
|
||||||
new SelectorDimFilter("quality", "business"),
|
new SelectorDimFilter("quality", "business", null),
|
||||||
new SelectorDimFilter("quality", "entertainment"),
|
new SelectorDimFilter("quality", "entertainment", null),
|
||||||
new SelectorDimFilter("quality", "health"),
|
new SelectorDimFilter("quality", "health", null),
|
||||||
new SelectorDimFilter("quality", "news"),
|
new SelectorDimFilter("quality", "news", null),
|
||||||
new SelectorDimFilter("quality", "technology"),
|
new SelectorDimFilter("quality", "technology", null),
|
||||||
new SelectorDimFilter("quality", "travel")
|
new SelectorDimFilter("quality", "travel", null)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
@ -3114,7 +3121,7 @@ public class GroupByQueryRunnerTest
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||||
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }"))
|
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }", null))
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
QueryRunnerTestHelper.rowsCount,
|
QueryRunnerTestHelper.rowsCount,
|
||||||
|
@ -3375,7 +3382,7 @@ public class GroupByQueryRunnerTest
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||||
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }"))
|
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }", null))
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
QueryRunnerTestHelper.rowsCount,
|
QueryRunnerTestHelper.rowsCount,
|
||||||
|
@ -3633,7 +3640,7 @@ public class GroupByQueryRunnerTest
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||||
.setDimFilter(new JavaScriptDimFilter("market", "function(dim){ return true; }"))
|
.setDimFilter(new JavaScriptDimFilter("market", "function(dim){ return true; }", null))
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
QueryRunnerTestHelper.rowsCount,
|
QueryRunnerTestHelper.rowsCount,
|
||||||
|
@ -4054,8 +4061,8 @@ public class GroupByQueryRunnerTest
|
||||||
.setDimFilter(
|
.setDimFilter(
|
||||||
new OrDimFilter(
|
new OrDimFilter(
|
||||||
Arrays.<DimFilter>asList(
|
Arrays.<DimFilter>asList(
|
||||||
new SelectorDimFilter("market", "spot"),
|
new SelectorDimFilter("market", "spot", null),
|
||||||
new SelectorDimFilter("market", "upfront")
|
new SelectorDimFilter("market", "upfront", null)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
@ -4286,7 +4293,7 @@ public class GroupByQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
||||||
.setDimFilter(new SelectorDimFilter("quality", "mezzanine"))
|
.setDimFilter(new SelectorDimFilter("quality", "mezzanine", null))
|
||||||
.setContext(ImmutableMap.<String, Object>of("bySegment", true));
|
.setContext(ImmutableMap.<String, Object>of("bySegment", true));
|
||||||
final GroupByQuery fullQuery = builder.build();
|
final GroupByQuery fullQuery = builder.build();
|
||||||
QueryToolChest toolChest = factory.getToolchest();
|
QueryToolChest toolChest = factory.getToolchest();
|
||||||
|
@ -4362,7 +4369,7 @@ public class GroupByQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
||||||
.setDimFilter(new SelectorDimFilter("quality", "mezzanine"))
|
.setDimFilter(new SelectorDimFilter("quality", "mezzanine", null))
|
||||||
.setContext(ImmutableMap.<String, Object>of("bySegment", true));
|
.setContext(ImmutableMap.<String, Object>of("bySegment", true));
|
||||||
final GroupByQuery fullQuery = builder.build();
|
final GroupByQuery fullQuery = builder.build();
|
||||||
QueryToolChest toolChest = factory.getToolchest();
|
QueryToolChest toolChest = factory.getToolchest();
|
||||||
|
@ -4437,7 +4444,7 @@ public class GroupByQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
||||||
.setDimFilter(new SelectorDimFilter("quality", "mezzanine"))
|
.setDimFilter(new SelectorDimFilter("quality", "mezzanine", null))
|
||||||
.setContext(ImmutableMap.<String, Object>of("bySegment", true));
|
.setContext(ImmutableMap.<String, Object>of("bySegment", true));
|
||||||
final GroupByQuery fullQuery = builder.build();
|
final GroupByQuery fullQuery = builder.build();
|
||||||
QueryToolChest toolChest = factory.getToolchest();
|
QueryToolChest toolChest = factory.getToolchest();
|
||||||
|
@ -4474,11 +4481,11 @@ public class GroupByQueryRunnerTest
|
||||||
|
|
||||||
List<DimFilter> dimFilters = Lists.<DimFilter>newArrayList(
|
List<DimFilter> dimFilters = Lists.<DimFilter>newArrayList(
|
||||||
new ExtractionDimFilter("quality", "automotiveAndBusinessAndNewsAndMezzanine", lookupExtractionFn, null),
|
new ExtractionDimFilter("quality", "automotiveAndBusinessAndNewsAndMezzanine", lookupExtractionFn, null),
|
||||||
new SelectorDimFilter("quality", "entertainment"),
|
new SelectorDimFilter("quality", "entertainment", null),
|
||||||
new SelectorDimFilter("quality", "health"),
|
new SelectorDimFilter("quality", "health", null),
|
||||||
new SelectorDimFilter("quality", "premium"),
|
new SelectorDimFilter("quality", "premium", null),
|
||||||
new SelectorDimFilter("quality", "technology"),
|
new SelectorDimFilter("quality", "technology", null),
|
||||||
new SelectorDimFilter("quality", "travel")
|
new SelectorDimFilter("quality", "travel", null)
|
||||||
);
|
);
|
||||||
|
|
||||||
GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
|
GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
@ -4751,7 +4758,8 @@ public class GroupByQueryRunnerTest
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test public void testGroupByWithExtractionDimFilterNullDims()
|
@Test
|
||||||
|
public void testGroupByWithExtractionDimFilterNullDims()
|
||||||
{
|
{
|
||||||
Map<String, String> extractionMap = new HashMap<>();
|
Map<String, String> extractionMap = new HashMap<>();
|
||||||
extractionMap.put("", "EMPTY");
|
extractionMap.put("", "EMPTY");
|
||||||
|
@ -4773,4 +4781,111 @@ public class GroupByQueryRunnerTest
|
||||||
Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
||||||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testBySegmentResultsWithAllFiltersWithExtractionFns()
|
||||||
|
{
|
||||||
|
int segmentCount = 32;
|
||||||
|
Result<BySegmentResultValue> singleSegmentResult = new Result<BySegmentResultValue>(
|
||||||
|
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||||
|
new BySegmentResultValueClass(
|
||||||
|
Arrays.asList(
|
||||||
|
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||||
|
"2011-04-01",
|
||||||
|
"alias",
|
||||||
|
"mezzanine",
|
||||||
|
"rows",
|
||||||
|
6L,
|
||||||
|
"idx",
|
||||||
|
4420L
|
||||||
|
)
|
||||||
|
), "testSegment", new Interval("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
List<Result> bySegmentResults = Lists.newArrayList();
|
||||||
|
for (int i = 0; i < segmentCount; i++) {
|
||||||
|
bySegmentResults.add(singleSegmentResult);
|
||||||
|
}
|
||||||
|
|
||||||
|
String extractionJsFn = "function(str) { return 'super-' + str; }";
|
||||||
|
String jsFn = "function(x) { return(x === 'super-mezzanine') }";
|
||||||
|
ExtractionFn extractionFn = new JavaScriptExtractionFn(extractionJsFn, false);
|
||||||
|
|
||||||
|
List<DimFilter> superFilterList = new ArrayList<>();
|
||||||
|
superFilterList.add(new SelectorDimFilter("quality", "super-mezzanine", extractionFn));
|
||||||
|
superFilterList.add(new InDimFilter("quality", Arrays.asList("not-super-mezzanine", "FOOBAR", "super-mezzanine"), extractionFn));
|
||||||
|
superFilterList.add(new BoundDimFilter("quality", "super-mezzanine", "super-mezzanine", false, false, true, extractionFn));
|
||||||
|
superFilterList.add(new RegexDimFilter("quality", "super-mezzanine", extractionFn));
|
||||||
|
superFilterList.add(new SearchQueryDimFilter("quality", new ContainsSearchQuerySpec("super-mezzanine", true), extractionFn));
|
||||||
|
superFilterList.add(new JavaScriptDimFilter("quality", jsFn, extractionFn));
|
||||||
|
DimFilter superFilter = new AndDimFilter(superFilterList);
|
||||||
|
|
||||||
|
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")
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
||||||
|
.setDimFilter(superFilter)
|
||||||
|
.setContext(ImmutableMap.<String, Object>of("bySegment", true));
|
||||||
|
final GroupByQuery fullQuery = builder.build();
|
||||||
|
QueryToolChest toolChest = factory.getToolchest();
|
||||||
|
|
||||||
|
List<QueryRunner<Row>> singleSegmentRunners = Lists.newArrayList();
|
||||||
|
for (int i = 0; i < segmentCount; i++) {
|
||||||
|
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
|
||||||
|
}
|
||||||
|
ExecutorService exec = Executors.newCachedThreadPool();
|
||||||
|
QueryRunner theRunner = toolChest.postMergeQueryDecoration(
|
||||||
|
new FinalizeResultsQueryRunner<>(
|
||||||
|
toolChest.mergeResults(factory.mergeRunners(Executors.newCachedThreadPool(), singleSegmentRunners)),
|
||||||
|
toolChest
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(fullQuery, Maps.newHashMap()), "");
|
||||||
|
exec.shutdownNow();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithAllFiltersOnNullDimsWithExtractionFns()
|
||||||
|
{
|
||||||
|
Map<String, String> extractionMap = new HashMap<>();
|
||||||
|
extractionMap.put("", "EMPTY");
|
||||||
|
extractionMap.put(null, "EMPTY");
|
||||||
|
|
||||||
|
MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false);
|
||||||
|
LookupExtractionFn extractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, true);
|
||||||
|
String jsFn = "function(x) { return(x === 'EMPTY') }";
|
||||||
|
|
||||||
|
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 RegexDimFilter("null_column", "EMPTY", extractionFn));
|
||||||
|
superFilterList.add(new SearchQueryDimFilter("null_column", new ContainsSearchQuerySpec("EMPTY", true), extractionFn));
|
||||||
|
superFilterList.add(new JavaScriptDimFilter("null_column", jsFn, extractionFn));
|
||||||
|
DimFilter superFilter = new AndDimFilter(superFilterList);
|
||||||
|
|
||||||
|
GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||||
|
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("null_column", "alias")))
|
||||||
|
.setAggregatorSpecs(
|
||||||
|
Arrays.asList(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")))
|
||||||
|
.setGranularity(QueryRunnerTestHelper.dayGran)
|
||||||
|
.setDimFilter(superFilter).build();
|
||||||
|
|
||||||
|
List<Row> expectedResults = Arrays
|
||||||
|
.asList(GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 13L, "idx", 6619L),
|
||||||
|
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", null, "rows", 13L, "idx", 5827L));
|
||||||
|
|
||||||
|
Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
||||||
|
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -385,8 +385,8 @@ public class SearchQueryRunnerTest
|
||||||
.filters(
|
.filters(
|
||||||
new AndDimFilter(
|
new AndDimFilter(
|
||||||
Arrays.<DimFilter>asList(
|
Arrays.<DimFilter>asList(
|
||||||
new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "total_market"),
|
new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "total_market", null),
|
||||||
new SelectorDimFilter(QueryRunnerTestHelper.qualityDimension, "mezzanine"))))
|
new SelectorDimFilter(QueryRunnerTestHelper.qualityDimension, "mezzanine", null))))
|
||||||
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
.dimensions(QueryRunnerTestHelper.qualityDimension)
|
.dimensions(QueryRunnerTestHelper.qualityDimension)
|
||||||
.query("a")
|
.query("a")
|
||||||
|
|
|
@ -398,7 +398,7 @@ public class SelectQueryRunnerTest
|
||||||
for (int[] param : new int[][]{{3, 3}, {0, 1}, {5, 5}, {2, 7}, {3, 0}}) {
|
for (int[] param : new int[][]{{3, 3}, {0, 1}, {5, 5}, {2, 7}, {3, 0}}) {
|
||||||
SelectQuery query = newTestQuery()
|
SelectQuery query = newTestQuery()
|
||||||
.intervals(I_0112_0114)
|
.intervals(I_0112_0114)
|
||||||
.filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot"))
|
.filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null))
|
||||||
.granularity(QueryRunnerTestHelper.dayGran)
|
.granularity(QueryRunnerTestHelper.dayGran)
|
||||||
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
|
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
|
||||||
.metrics(Lists.<String>newArrayList(QueryRunnerTestHelper.indexMetric))
|
.metrics(Lists.<String>newArrayList(QueryRunnerTestHelper.indexMetric))
|
||||||
|
@ -463,8 +463,8 @@ public class SelectQueryRunnerTest
|
||||||
.filters(
|
.filters(
|
||||||
new AndDimFilter(
|
new AndDimFilter(
|
||||||
Arrays.<DimFilter>asList(
|
Arrays.<DimFilter>asList(
|
||||||
new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot"),
|
new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null),
|
||||||
new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "foo")
|
new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "foo", null)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
|
@ -791,7 +791,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.granularity(QueryRunnerTestHelper.dayGran)
|
.granularity(QueryRunnerTestHelper.dayGran)
|
||||||
.filters(new RegexDimFilter(QueryRunnerTestHelper.marketDimension, "^.p.*$")) // spot and upfront
|
.filters(new RegexDimFilter(QueryRunnerTestHelper.marketDimension, "^.p.*$", null)) // spot and upfront
|
||||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||||
.aggregators(
|
.aggregators(
|
||||||
Arrays.<AggregatorFactory>asList(
|
Arrays.<AggregatorFactory>asList(
|
||||||
|
@ -1232,12 +1232,14 @@ public class TimeseriesQueryRunnerTest
|
||||||
.granularity(QueryRunnerTestHelper.dayGran)
|
.granularity(QueryRunnerTestHelper.dayGran)
|
||||||
.filters(
|
.filters(
|
||||||
new InDimFilter(
|
new InDimFilter(
|
||||||
QueryRunnerTestHelper.marketDimension, Arrays.asList(
|
QueryRunnerTestHelper.marketDimension,
|
||||||
"spot",
|
Arrays.asList(
|
||||||
"upfront",
|
"spot",
|
||||||
"total_market",
|
"upfront",
|
||||||
"billyblank"
|
"total_market",
|
||||||
)
|
"billyblank"
|
||||||
|
),
|
||||||
|
null
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||||
|
@ -1466,7 +1468,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.granularity(QueryRunnerTestHelper.dayGran)
|
.granularity(QueryRunnerTestHelper.dayGran)
|
||||||
.filters(new NotDimFilter(new SelectorDimFilter("bobby", "sally")))
|
.filters(new NotDimFilter(new SelectorDimFilter("bobby", "sally", null)))
|
||||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
|
@ -2198,6 +2200,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
"spot",
|
"spot",
|
||||||
true,
|
true,
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
new BoundDimFilter(
|
new BoundDimFilter(
|
||||||
|
@ -2206,6 +2209,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
"spotify",
|
"spotify",
|
||||||
null,
|
null,
|
||||||
true,
|
true,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
(DimFilter) new BoundDimFilter(
|
(DimFilter) new BoundDimFilter(
|
||||||
|
@ -2214,6 +2218,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
"spot",
|
"spot",
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
|
@ -3042,7 +3042,7 @@ public class TopNQueryRunnerTest
|
||||||
.granularity(QueryRunnerTestHelper.allGran)
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
.dimension("null_column")
|
.dimension("null_column")
|
||||||
.filters(
|
.filters(
|
||||||
new SelectorDimFilter("null_column", null)
|
new SelectorDimFilter("null_column", null, null)
|
||||||
)
|
)
|
||||||
.metric(QueryRunnerTestHelper.indexMetric)
|
.metric(QueryRunnerTestHelper.indexMetric)
|
||||||
.threshold(4)
|
.threshold(4)
|
||||||
|
@ -3127,7 +3127,7 @@ public class TopNQueryRunnerTest
|
||||||
.granularity(QueryGranularity.ALL)
|
.granularity(QueryGranularity.ALL)
|
||||||
.dimension("partial_null_column")
|
.dimension("partial_null_column")
|
||||||
.metric(QueryRunnerTestHelper.uniqueMetric)
|
.metric(QueryRunnerTestHelper.uniqueMetric)
|
||||||
.filters(new SelectorDimFilter("partial_null_column", null))
|
.filters(new SelectorDimFilter("partial_null_column", null, null))
|
||||||
.threshold(1000)
|
.threshold(1000)
|
||||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
|
@ -3159,7 +3159,7 @@ public class TopNQueryRunnerTest
|
||||||
.granularity(QueryGranularity.ALL)
|
.granularity(QueryGranularity.ALL)
|
||||||
.dimension("partial_null_column")
|
.dimension("partial_null_column")
|
||||||
.metric(QueryRunnerTestHelper.uniqueMetric)
|
.metric(QueryRunnerTestHelper.uniqueMetric)
|
||||||
.filters(new SelectorDimFilter("partial_null_column", "value"))
|
.filters(new SelectorDimFilter("partial_null_column", "value", null))
|
||||||
.threshold(1000)
|
.threshold(1000)
|
||||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
|
|
|
@ -29,6 +29,8 @@ import io.druid.data.input.impl.InputRowParser;
|
||||||
import io.druid.data.input.impl.MapInputRowParser;
|
import io.druid.data.input.impl.MapInputRowParser;
|
||||||
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||||
import io.druid.data.input.impl.TimestampSpec;
|
import io.druid.data.input.impl.TimestampSpec;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||||
import io.druid.query.filter.BoundDimFilter;
|
import io.druid.query.filter.BoundDimFilter;
|
||||||
import io.druid.query.filter.DimFilter;
|
import io.druid.query.filter.DimFilter;
|
||||||
import io.druid.segment.IndexBuilder;
|
import io.druid.segment.IndexBuilder;
|
||||||
|
@ -78,10 +80,10 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testLexicographicMatchEverything()
|
public void testLexicographicMatchEverything()
|
||||||
{
|
{
|
||||||
final List<BoundDimFilter> filters = ImmutableList.of(
|
final List<BoundDimFilter> filters = ImmutableList.of(
|
||||||
new BoundDimFilter("dim0", "", "z", false, false, false),
|
new BoundDimFilter("dim0", "", "z", false, false, false, null),
|
||||||
new BoundDimFilter("dim1", "", "z", false, false, false),
|
new BoundDimFilter("dim1", "", "z", false, false, false, null),
|
||||||
new BoundDimFilter("dim2", "", "z", false, false, false),
|
new BoundDimFilter("dim2", "", "z", false, false, false, null),
|
||||||
new BoundDimFilter("dim3", "", "z", false, false, false)
|
new BoundDimFilter("dim3", "", "z", false, false, false, null)
|
||||||
);
|
);
|
||||||
|
|
||||||
for (BoundDimFilter filter : filters) {
|
for (BoundDimFilter filter : filters) {
|
||||||
|
@ -93,15 +95,15 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testLexicographicMatchNull()
|
public void testLexicographicMatchNull()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim0", "", "", false, false, false),
|
new BoundDimFilter("dim0", "", "", false, false, false, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "", "", false, false, false),
|
new BoundDimFilter("dim1", "", "", false, false, false, null),
|
||||||
ImmutableList.of("0")
|
ImmutableList.of("0")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim2", "", "", false, false, false),
|
new BoundDimFilter("dim2", "", "", false, false, false, null),
|
||||||
ImmutableList.of("1", "2", "5")
|
ImmutableList.of("1", "2", "5")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -110,27 +112,27 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testLexicographicMatchMissingColumn()
|
public void testLexicographicMatchMissingColumn()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim3", "", "", false, false, false),
|
new BoundDimFilter("dim3", "", "", false, false, false, null),
|
||||||
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim3", "", "", true, false, false),
|
new BoundDimFilter("dim3", "", "", true, false, false, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim3", "", "", false, true, false),
|
new BoundDimFilter("dim3", "", "", false, true, false, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim3", "", null, false, true, false),
|
new BoundDimFilter("dim3", "", null, false, true, false, null),
|
||||||
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim3", null, "", false, false, false),
|
new BoundDimFilter("dim3", null, "", false, false, false, null),
|
||||||
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim3", null, "", false, true, false),
|
new BoundDimFilter("dim3", null, "", false, true, false, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -139,15 +141,15 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testLexicographicMatchTooStrict()
|
public void testLexicographicMatchTooStrict()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "abc", "abc", true, false, false),
|
new BoundDimFilter("dim1", "abc", "abc", true, false, false, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "abc", "abc", true, true, false),
|
new BoundDimFilter("dim1", "abc", "abc", true, true, false, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "abc", "abc", false, true, false),
|
new BoundDimFilter("dim1", "abc", "abc", false, true, false, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -156,7 +158,7 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testLexicographicMatchExactlySingleValue()
|
public void testLexicographicMatchExactlySingleValue()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "abc", "abc", false, false, false),
|
new BoundDimFilter("dim1", "abc", "abc", false, false, false, null),
|
||||||
ImmutableList.of("5")
|
ImmutableList.of("5")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -165,7 +167,7 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testLexicographicMatchSurroundingSingleValue()
|
public void testLexicographicMatchSurroundingSingleValue()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "ab", "abd", true, true, false),
|
new BoundDimFilter("dim1", "ab", "abd", true, true, false, null),
|
||||||
ImmutableList.of("5")
|
ImmutableList.of("5")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -174,7 +176,7 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testLexicographicMatchNoUpperLimit()
|
public void testLexicographicMatchNoUpperLimit()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "ab", null, true, true, false),
|
new BoundDimFilter("dim1", "ab", null, true, true, false, null),
|
||||||
ImmutableList.of("4", "5")
|
ImmutableList.of("4", "5")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -183,7 +185,7 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testLexicographicMatchNoLowerLimit()
|
public void testLexicographicMatchNoLowerLimit()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", null, "abd", true, true, false),
|
new BoundDimFilter("dim1", null, "abd", true, true, false, null),
|
||||||
ImmutableList.of("0", "1", "2", "3", "5")
|
ImmutableList.of("0", "1", "2", "3", "5")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -192,11 +194,11 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testLexicographicMatchNumbers()
|
public void testLexicographicMatchNumbers()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "1", "3", false, false, false),
|
new BoundDimFilter("dim1", "1", "3", false, false, false, null),
|
||||||
ImmutableList.of("1", "2", "3")
|
ImmutableList.of("1", "2", "3")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "1", "3", true, true, false),
|
new BoundDimFilter("dim1", "1", "3", true, true, false, null),
|
||||||
ImmutableList.of("1", "2")
|
ImmutableList.of("1", "2")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -205,19 +207,19 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testAlphaNumericMatchNull()
|
public void testAlphaNumericMatchNull()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim0", "", "", false, false, true),
|
new BoundDimFilter("dim0", "", "", false, false, true, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "", "", false, false, true),
|
new BoundDimFilter("dim1", "", "", false, false, true, null),
|
||||||
ImmutableList.of("0")
|
ImmutableList.of("0")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim2", "", "", false, false, true),
|
new BoundDimFilter("dim2", "", "", false, false, true, null),
|
||||||
ImmutableList.of("1", "2", "5")
|
ImmutableList.of("1", "2", "5")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim3", "", "", false, false, true),
|
new BoundDimFilter("dim3", "", "", false, false, true, null),
|
||||||
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -226,15 +228,15 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testAlphaNumericMatchTooStrict()
|
public void testAlphaNumericMatchTooStrict()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "2", "2", true, false, true),
|
new BoundDimFilter("dim1", "2", "2", true, false, true, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "2", "2", true, true, true),
|
new BoundDimFilter("dim1", "2", "2", true, true, true, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "2", "2", false, true, true),
|
new BoundDimFilter("dim1", "2", "2", false, true, true, null),
|
||||||
ImmutableList.<String>of()
|
ImmutableList.<String>of()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -243,7 +245,7 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testAlphaNumericMatchExactlySingleValue()
|
public void testAlphaNumericMatchExactlySingleValue()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "2", "2", false, false, true),
|
new BoundDimFilter("dim1", "2", "2", false, false, true, null),
|
||||||
ImmutableList.of("2")
|
ImmutableList.of("2")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -252,7 +254,7 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testAlphaNumericMatchSurroundingSingleValue()
|
public void testAlphaNumericMatchSurroundingSingleValue()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "1", "3", true, true, true),
|
new BoundDimFilter("dim1", "1", "3", true, true, true, null),
|
||||||
ImmutableList.of("2")
|
ImmutableList.of("2")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -261,7 +263,7 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testAlphaNumericMatchNoUpperLimit()
|
public void testAlphaNumericMatchNoUpperLimit()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", "1", null, true, true, true),
|
new BoundDimFilter("dim1", "1", null, true, true, true, null),
|
||||||
ImmutableList.of("1", "2", "4", "5")
|
ImmutableList.of("1", "2", "4", "5")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -270,11 +272,56 @@ public class BoundFilterTest extends BaseFilterTest
|
||||||
public void testAlphaNumericMatchNoLowerLimit()
|
public void testAlphaNumericMatchNoLowerLimit()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new BoundDimFilter("dim1", null, "2", true, true, true),
|
new BoundDimFilter("dim1", null, "2", true, true, true, null),
|
||||||
ImmutableList.of("0", "3")
|
ImmutableList.of("0", "3")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMatchWithExtractionFn()
|
||||||
|
{
|
||||||
|
String extractionJsFn = "function(str) { return 'super-' + str; }";
|
||||||
|
ExtractionFn superFn = new JavaScriptExtractionFn(extractionJsFn, false);
|
||||||
|
|
||||||
|
String nullJsFn = "function(str) { return null; }";
|
||||||
|
ExtractionFn makeNullFn = new JavaScriptExtractionFn(nullJsFn, false);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new BoundDimFilter("dim0", "", "", false, false, false, makeNullFn),
|
||||||
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new BoundDimFilter("dim1", "super-ab", "super-abd", true, true, false, superFn),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new BoundDimFilter("dim1", "super-0", "super-10", false, false, true, superFn),
|
||||||
|
ImmutableList.of("1", "2", "3")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new BoundDimFilter("dim2", "super-", "super-zzzzzz", false, false, false, superFn),
|
||||||
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new BoundDimFilter("dim2", "super-null", "super-null", false, false, false, superFn),
|
||||||
|
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")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new BoundDimFilter("dim4", "super-null", "super-null", false, false, false, superFn),
|
||||||
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
private void assertFilterMatches(
|
private void assertFilterMatches(
|
||||||
final DimFilter filter,
|
final DimFilter filter,
|
||||||
final List<String> expectedRows
|
final List<String> expectedRows
|
||||||
|
|
|
@ -160,7 +160,7 @@ public class ExtractionDimFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testEmpty()
|
public void testEmpty()
|
||||||
{
|
{
|
||||||
ExtractionFilter extractionFilter = new ExtractionFilter(
|
SelectorFilter extractionFilter = new SelectorFilter(
|
||||||
"foo", "NFDJUKFNDSJFNS", DIM_EXTRACTION_FN
|
"foo", "NFDJUKFNDSJFNS", DIM_EXTRACTION_FN
|
||||||
);
|
);
|
||||||
ImmutableBitmap immutableBitmap = extractionFilter.getBitmapIndex(BITMAP_INDEX_SELECTOR);
|
ImmutableBitmap immutableBitmap = extractionFilter.getBitmapIndex(BITMAP_INDEX_SELECTOR);
|
||||||
|
@ -170,7 +170,7 @@ public class ExtractionDimFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testNull()
|
public void testNull()
|
||||||
{
|
{
|
||||||
ExtractionFilter extractionFilter = new ExtractionFilter(
|
SelectorFilter extractionFilter = new SelectorFilter(
|
||||||
"FDHJSFFHDS", "extractDimVal", DIM_EXTRACTION_FN
|
"FDHJSFFHDS", "extractDimVal", DIM_EXTRACTION_FN
|
||||||
);
|
);
|
||||||
ImmutableBitmap immutableBitmap = extractionFilter.getBitmapIndex(BITMAP_INDEX_SELECTOR);
|
ImmutableBitmap immutableBitmap = extractionFilter.getBitmapIndex(BITMAP_INDEX_SELECTOR);
|
||||||
|
@ -180,7 +180,7 @@ public class ExtractionDimFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testNormal()
|
public void testNormal()
|
||||||
{
|
{
|
||||||
ExtractionFilter extractionFilter = new ExtractionFilter(
|
SelectorFilter extractionFilter = new SelectorFilter(
|
||||||
"foo", "extractDimVal", DIM_EXTRACTION_FN
|
"foo", "extractDimVal", DIM_EXTRACTION_FN
|
||||||
);
|
);
|
||||||
ImmutableBitmap immutableBitmap = extractionFilter.getBitmapIndex(BITMAP_INDEX_SELECTOR);
|
ImmutableBitmap immutableBitmap = extractionFilter.getBitmapIndex(BITMAP_INDEX_SELECTOR);
|
||||||
|
|
|
@ -32,6 +32,9 @@ import io.druid.data.input.impl.InputRowParser;
|
||||||
import io.druid.data.input.impl.MapInputRowParser;
|
import io.druid.data.input.impl.MapInputRowParser;
|
||||||
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||||
import io.druid.data.input.impl.TimestampSpec;
|
import io.druid.data.input.impl.TimestampSpec;
|
||||||
|
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.filter.DimFilter;
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
import io.druid.query.filter.InDimFilter;
|
import io.druid.query.filter.InDimFilter;
|
||||||
|
@ -46,6 +49,7 @@ import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -100,63 +104,177 @@ public class InFilterTest extends BaseFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testSingleValueStringColumnWithoutNulls()
|
public void testSingleValueStringColumnWithoutNulls()
|
||||||
{
|
{
|
||||||
Assert.assertEquals(ImmutableList.<Integer>of(), select(toInFilter("dim0", null)));
|
assertFilterMatches(
|
||||||
Assert.assertEquals(ImmutableList.<Integer>of(), select(toInFilter("dim0", "", "")));
|
toInFilter("dim0", null),
|
||||||
Assert.assertEquals(ImmutableList.of(0, 2), select(toInFilter("dim0", "a", "c")));
|
ImmutableList.<String>of()
|
||||||
Assert.assertEquals(ImmutableList.of(4), select(toInFilter("dim0", "e", "x")));
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim0", "", ""),
|
||||||
|
ImmutableList.<String>of()
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim0", "a", "c"),
|
||||||
|
ImmutableList.of("a", "c")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim0", "e", "x"),
|
||||||
|
ImmutableList.of("e")
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSingleValueStringColumnWithNulls()
|
public void testSingleValueStringColumnWithNulls()
|
||||||
{
|
{
|
||||||
Assert.assertEquals(ImmutableList.of(0), select(toInFilter("dim1", null, "")));
|
assertFilterMatches(
|
||||||
Assert.assertEquals(ImmutableList.of(0), select(toInFilter("dim1", "")));
|
toInFilter("dim1", null, ""),
|
||||||
Assert.assertEquals(ImmutableList.of(0, 1, 5), select(toInFilter("dim1", null, "10", "abc")));
|
ImmutableList.of("a")
|
||||||
Assert.assertEquals(ImmutableList.<Integer>of(), select(toInFilter("dim1", "-1", "ab", "de")));
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim1", ""),
|
||||||
|
ImmutableList.of("a")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim1", null, "10", "abc"),
|
||||||
|
ImmutableList.of("a", "b", "f")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim1", "-1", "ab", "de"),
|
||||||
|
ImmutableList.<String>of()
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMultiValueStringColumn()
|
public void testMultiValueStringColumn()
|
||||||
{
|
{
|
||||||
Assert.assertEquals(ImmutableList.of(1, 2, 5), select(toInFilter("dim2", null)));
|
assertFilterMatches(
|
||||||
Assert.assertEquals(ImmutableList.of(1, 2, 5), select(toInFilter("dim2", "", (String)null)));
|
toInFilter("dim2", null),
|
||||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 5), select(toInFilter("dim2", null, "a")));
|
ImmutableList.of("b", "c", "f")
|
||||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 5), select(toInFilter("dim2", null, "b")));
|
);
|
||||||
Assert.assertEquals(ImmutableList.of(4), select(toInFilter("dim2", "c")));
|
|
||||||
Assert.assertEquals(ImmutableList.<Integer>of(), select(toInFilter("dim2", "d")));
|
assertFilterMatches(
|
||||||
|
toInFilter("dim2", "", (String)null),
|
||||||
|
ImmutableList.of("b", "c", "f")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim2", null, "a"),
|
||||||
|
ImmutableList.of("a", "b", "c", "d", "f")
|
||||||
|
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim2", null, "b"),
|
||||||
|
ImmutableList.of("a", "b", "c", "f")
|
||||||
|
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim2", "c"),
|
||||||
|
ImmutableList.of("e")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim2", "d"),
|
||||||
|
ImmutableList.<String>of()
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMissingColumn()
|
public void testMissingColumn()
|
||||||
{
|
{
|
||||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(toInFilter("dim3", null, (String)null)));
|
assertFilterMatches(
|
||||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(toInFilter("dim3", "")));
|
toInFilter("dim3", null, (String)null),
|
||||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(toInFilter("dim3", null, "a")));
|
ImmutableList.of("a", "b", "c", "d", "e", "f")
|
||||||
Assert.assertEquals(ImmutableList.<Integer>of(), select(toInFilter("dim3", "a")));
|
);
|
||||||
Assert.assertEquals(ImmutableList.<Integer>of(), select(toInFilter("dim3", "b")));
|
|
||||||
Assert.assertEquals(ImmutableList.<Integer>of(), select(toInFilter("dim3", "c")));
|
assertFilterMatches(
|
||||||
|
toInFilter("dim3", ""),
|
||||||
|
ImmutableList.of("a", "b", "c", "d", "e", "f")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim3", null, "a"),
|
||||||
|
ImmutableList.of("a", "b", "c", "d", "e", "f")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim3", "a"),
|
||||||
|
ImmutableList.<String>of()
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim3", "b"),
|
||||||
|
ImmutableList.<String>of()
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilter("dim3", "c"),
|
||||||
|
ImmutableList.<String>of()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMatchWithExtractionFn()
|
||||||
|
{
|
||||||
|
String extractionJsFn = "function(str) { return 'super-' + str; }";
|
||||||
|
ExtractionFn superFn = new JavaScriptExtractionFn(extractionJsFn, false);
|
||||||
|
|
||||||
|
String nullJsFn = "function(str) { if (str === null) { return 'YES'; } else { return 'NO';} }";
|
||||||
|
ExtractionFn yesNullFn = new JavaScriptExtractionFn(nullJsFn, false);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b"),
|
||||||
|
ImmutableList.of("a", "b", "c", "d", "f")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilterWithFn("dim2", yesNullFn, "YES"),
|
||||||
|
ImmutableList.of("b", "c", "f")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilterWithFn("dim1", superFn, "super-null", "super-10", "super-def"),
|
||||||
|
ImmutableList.of("a", "b", "e")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilterWithFn("dim3", yesNullFn, "NO"),
|
||||||
|
ImmutableList.<String>of()
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilterWithFn("dim3", yesNullFn, "YES"),
|
||||||
|
ImmutableList.of("a", "b", "c", "d", "e", "f")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
toInFilterWithFn("dim1", yesNullFn, "NO"),
|
||||||
|
ImmutableList.of("b", "c", "d", "e", "f")
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private DimFilter toInFilter(String dim, String value, String... values)
|
private DimFilter toInFilter(String dim, String value, String... values)
|
||||||
{
|
{
|
||||||
return new InDimFilter(dim, Lists.asList(value, values));
|
return new InDimFilter(dim, Lists.asList(value, values), null);
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<Integer> select(final DimFilter filter)
|
private DimFilter toInFilterWithFn(String dim, ExtractionFn fn, String value, String... values)
|
||||||
{
|
{
|
||||||
return Lists.newArrayList(
|
return new InDimFilter(dim, Lists.asList(value, values), fn);
|
||||||
Iterables.transform(
|
}
|
||||||
selectColumnValuesMatchingFilter(filter, "dim0"),
|
|
||||||
new Function<String, Integer>()
|
private void assertFilterMatches(
|
||||||
{
|
final DimFilter filter,
|
||||||
@Override
|
final List<String> expectedRows
|
||||||
public Integer apply(String input)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(input.length() == 1);
|
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||||
return ((int) input.charAt(0)) - ((int) 'a');
|
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||||
}
|
|
||||||
}
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,179 @@
|
||||||
|
/*
|
||||||
|
* 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.segment.filter;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import com.metamx.common.Pair;
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
|
import io.druid.data.input.impl.DimensionsSpec;
|
||||||
|
import io.druid.data.input.impl.InputRowParser;
|
||||||
|
import io.druid.data.input.impl.MapInputRowParser;
|
||||||
|
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||||
|
import io.druid.data.input.impl.TimestampSpec;
|
||||||
|
import io.druid.query.extraction.MapLookupExtractor;
|
||||||
|
import io.druid.query.filter.DimFilter;
|
||||||
|
import io.druid.query.filter.JavaScriptDimFilter;
|
||||||
|
import io.druid.query.filter.SelectorDimFilter;
|
||||||
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
|
import io.druid.query.lookup.LookupExtractor;
|
||||||
|
import io.druid.segment.IndexBuilder;
|
||||||
|
import io.druid.segment.StorageAdapter;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
public class JavascriptFilterTest extends BaseFilterTest
|
||||||
|
{
|
||||||
|
private static final String TIMESTAMP_COLUMN = "timestamp";
|
||||||
|
|
||||||
|
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
|
||||||
|
new TimeAndDimsParseSpec(
|
||||||
|
new TimestampSpec(TIMESTAMP_COLUMN, "iso", new DateTime("2000")),
|
||||||
|
new DimensionsSpec(
|
||||||
|
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "0", "dim1", "", "dim2", ImmutableList.of("a", "b"))),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "1", "dim1", "10", "dim2", ImmutableList.of())),
|
||||||
|
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"))
|
||||||
|
);
|
||||||
|
|
||||||
|
public JavascriptFilterTest(
|
||||||
|
String testName,
|
||||||
|
IndexBuilder indexBuilder,
|
||||||
|
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||||
|
boolean optimize
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(ROWS, indexBuilder, finisher, optimize);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final String jsNullFilter = "function(x) { return(x === null) }";
|
||||||
|
|
||||||
|
private String jsValueFilter(String value)
|
||||||
|
{
|
||||||
|
String jsFn = "function(x) { return(x === '" + value + "') }";
|
||||||
|
return jsFn;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSingleValueStringColumnWithoutNulls()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim0", jsNullFilter, null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim0", jsValueFilter(""), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim0", jsValueFilter("0"), null), ImmutableList.of("0"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim0", jsValueFilter("1"), null), ImmutableList.of("1"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSingleValueStringColumnWithNulls()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim1", jsNullFilter, null), ImmutableList.of("0"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim1", jsValueFilter("10"), null), ImmutableList.of("1"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim1", jsValueFilter("2"), null), ImmutableList.of("2"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim1", jsValueFilter("1"), null), ImmutableList.of("3"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim1", jsValueFilter("def"), null), ImmutableList.of("4"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim1", jsValueFilter("abc"), null), ImmutableList.of("5"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim1", jsValueFilter("ab"), null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMultiValueStringColumn()
|
||||||
|
{
|
||||||
|
// multi-val null......
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim2", jsNullFilter, null), ImmutableList.of("1", "2", "5"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim2", jsValueFilter("a"), null), ImmutableList.of("0", "3"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim2", jsValueFilter("b"), null), ImmutableList.of("0"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim2", jsValueFilter("c"), null), ImmutableList.of("4"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim2", jsValueFilter("d"), null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMissingColumnSpecifiedInDimensionList()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim3", jsNullFilter, null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim3", jsValueFilter("a"), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim3", jsValueFilter("b"), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim3", jsValueFilter("c"), null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMissingColumnNotSpecifiedInDimensionList()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim4", jsNullFilter, null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim4", jsValueFilter("a"), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim4", jsValueFilter("b"), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim4", jsValueFilter("c"), null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testJavascriptFilterWithLookupExtractionFn()
|
||||||
|
{
|
||||||
|
final Map<String, String> stringMap = ImmutableMap.of(
|
||||||
|
"1", "HELLO",
|
||||||
|
"a", "HELLO",
|
||||||
|
"def", "HELLO",
|
||||||
|
"abc", "UNKNOWN"
|
||||||
|
);
|
||||||
|
LookupExtractor mapExtractor = new MapLookupExtractor(stringMap, false);
|
||||||
|
LookupExtractionFn lookupFn = new LookupExtractionFn(mapExtractor, false, "UNKNOWN", false, true);
|
||||||
|
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim0", jsValueFilter("HELLO"), lookupFn), ImmutableList.of("1"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim0", jsValueFilter("UNKNOWN"), lookupFn), ImmutableList.of("0", "2", "3", "4", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim1", jsValueFilter("HELLO"), lookupFn), ImmutableList.of("3", "4"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim1", jsValueFilter("UNKNOWN"), lookupFn), ImmutableList.of("0", "1", "2", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim2", jsValueFilter("HELLO"), lookupFn), ImmutableList.of("0", "3"));
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim2", jsValueFilter("UNKNOWN"), lookupFn), ImmutableList.of("0", "1", "2", "4", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim3", jsValueFilter("HELLO"), lookupFn), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim3", jsValueFilter("UNKNOWN"), lookupFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim4", jsValueFilter("HELLO"), lookupFn), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new JavaScriptDimFilter("dim4", jsValueFilter("UNKNOWN"), lookupFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertFilterMatches(
|
||||||
|
final DimFilter filter,
|
||||||
|
final List<String> expectedRows
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||||
|
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||||
|
}
|
||||||
|
}
|
|
@ -79,19 +79,19 @@ public class NotFilterTest extends BaseFilterTest
|
||||||
public void testNotSelector()
|
public void testNotSelector()
|
||||||
{
|
{
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new NotDimFilter(new SelectorDimFilter("dim0", null)),
|
new NotDimFilter(new SelectorDimFilter("dim0", null, null)),
|
||||||
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new NotDimFilter(new SelectorDimFilter("dim0", "")),
|
new NotDimFilter(new SelectorDimFilter("dim0", "", null)),
|
||||||
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new NotDimFilter(new SelectorDimFilter("dim0", "0")),
|
new NotDimFilter(new SelectorDimFilter("dim0", "0", null)),
|
||||||
ImmutableList.of("1", "2", "3", "4", "5")
|
ImmutableList.of("1", "2", "3", "4", "5")
|
||||||
);
|
);
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new NotDimFilter(new SelectorDimFilter("dim0", "1")),
|
new NotDimFilter(new SelectorDimFilter("dim0", "1", null)),
|
||||||
ImmutableList.of("0", "2", "3", "4", "5")
|
ImmutableList.of("0", "2", "3", "4", "5")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,159 @@
|
||||||
|
/*
|
||||||
|
* 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.segment.filter;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import com.metamx.common.Pair;
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
|
import io.druid.data.input.impl.DimensionsSpec;
|
||||||
|
import io.druid.data.input.impl.InputRowParser;
|
||||||
|
import io.druid.data.input.impl.MapInputRowParser;
|
||||||
|
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||||
|
import io.druid.data.input.impl.TimestampSpec;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||||
|
import io.druid.query.filter.DimFilter;
|
||||||
|
import io.druid.query.filter.RegexDimFilter;
|
||||||
|
import io.druid.segment.IndexBuilder;
|
||||||
|
import io.druid.segment.StorageAdapter;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
public class RegexFilterTest extends BaseFilterTest
|
||||||
|
{
|
||||||
|
private static final String TIMESTAMP_COLUMN = "timestamp";
|
||||||
|
|
||||||
|
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
|
||||||
|
new TimeAndDimsParseSpec(
|
||||||
|
new TimestampSpec(TIMESTAMP_COLUMN, "iso", new DateTime("2000")),
|
||||||
|
new DimensionsSpec(
|
||||||
|
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "0", "dim1", "", "dim2", ImmutableList.of("a", "b"))),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "1", "dim1", "10", "dim2", ImmutableList.of())),
|
||||||
|
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", "abdef", "dim2", ImmutableList.of("c"))),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "5", "dim1", "abc"))
|
||||||
|
);
|
||||||
|
|
||||||
|
public RegexFilterTest(
|
||||||
|
String testName,
|
||||||
|
IndexBuilder indexBuilder,
|
||||||
|
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||||
|
boolean optimize
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(ROWS, indexBuilder, finisher, optimize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSingleValueStringColumnWithoutNulls()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim0", ".*", null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim0", "0", null), ImmutableList.of("0"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim0", "5", null), ImmutableList.of("5"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSingleValueStringColumnWithNulls()
|
||||||
|
{
|
||||||
|
// RegexFilter always returns false for null row values.
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim1", ".*", null), ImmutableList.of("1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim1", "10", null), ImmutableList.of("1"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim1", "2", null), ImmutableList.of("2"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim1", "1", null), ImmutableList.of("1", "3"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim1", ".*def", null), ImmutableList.of("4"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim1", "abc", null), ImmutableList.of("5"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim1", "ab.*", null), ImmutableList.<String>of("4", "5"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMultiValueStringColumn()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim2", ".*", null), ImmutableList.of("0", "3", "4"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim2", "a", null), ImmutableList.of("0", "3"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim2", "b", null), ImmutableList.of("0"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim2", "c", null), ImmutableList.of("4"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim2", "d", null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMissingColumnSpecifiedInDimensionList()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim3", "", null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim3", "a", null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim3", "b", null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim3", "c", null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMissingColumnNotSpecifiedInDimensionList()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim4", "", null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim4", "a", null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim4", "b", null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim4", "c", null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRegexWithExtractionFn()
|
||||||
|
{
|
||||||
|
String nullJsFn = "function(str) { if (str === null) { return 'NOT_NULL_ANYMORE'; } else { return str;} }";
|
||||||
|
ExtractionFn changeNullFn = new JavaScriptExtractionFn(nullJsFn, false);
|
||||||
|
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim1", ".*ANYMORE", changeNullFn), ImmutableList.of("0"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim1", "ab.*", changeNullFn), ImmutableList.<String>of("4", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim2", ".*ANYMORE", changeNullFn), ImmutableList.of("1", "2", "5"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim2", "a.*", changeNullFn), ImmutableList.of("0", "3"));
|
||||||
|
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim3", ".*ANYMORE", changeNullFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim3", "a.*", changeNullFn), ImmutableList.<String>of());
|
||||||
|
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim4", ".*ANYMORE", changeNullFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new RegexDimFilter("dim4", "a.*", changeNullFn), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertFilterMatches(
|
||||||
|
final DimFilter filter,
|
||||||
|
final List<String> expectedRows
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||||
|
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,169 @@
|
||||||
|
/*
|
||||||
|
* 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.segment.filter;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import com.metamx.common.Pair;
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
|
import io.druid.data.input.impl.DimensionsSpec;
|
||||||
|
import io.druid.data.input.impl.InputRowParser;
|
||||||
|
import io.druid.data.input.impl.MapInputRowParser;
|
||||||
|
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||||
|
import io.druid.data.input.impl.TimestampSpec;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||||
|
import io.druid.query.filter.DimFilter;
|
||||||
|
import io.druid.query.filter.RegexDimFilter;
|
||||||
|
import io.druid.query.filter.SearchQueryDimFilter;
|
||||||
|
import io.druid.query.search.search.ContainsSearchQuerySpec;
|
||||||
|
import io.druid.query.search.search.SearchQuerySpec;
|
||||||
|
import io.druid.segment.IndexBuilder;
|
||||||
|
import io.druid.segment.StorageAdapter;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
public class SearchQueryFilterTest extends BaseFilterTest
|
||||||
|
{
|
||||||
|
private static final String TIMESTAMP_COLUMN = "timestamp";
|
||||||
|
|
||||||
|
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
|
||||||
|
new TimeAndDimsParseSpec(
|
||||||
|
new TimestampSpec(TIMESTAMP_COLUMN, "iso", new DateTime("2000")),
|
||||||
|
new DimensionsSpec(
|
||||||
|
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "0", "dim1", "", "dim2", ImmutableList.of("a", "b"))),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "1", "dim1", "10", "dim2", ImmutableList.of())),
|
||||||
|
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", "abdef", "dim2", ImmutableList.of("c"))),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "5", "dim1", "abc"))
|
||||||
|
);
|
||||||
|
|
||||||
|
public SearchQueryFilterTest(
|
||||||
|
String testName,
|
||||||
|
IndexBuilder indexBuilder,
|
||||||
|
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||||
|
boolean optimize
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(ROWS, indexBuilder, finisher, optimize);
|
||||||
|
}
|
||||||
|
|
||||||
|
private SearchQuerySpec specForValue(String value)
|
||||||
|
{
|
||||||
|
return new ContainsSearchQuerySpec(value, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSingleValueStringColumnWithoutNulls()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim0", specForValue(""), null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim0", specForValue("0"), null), ImmutableList.of("0"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim0", specForValue("5"), null), ImmutableList.of("5"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSingleValueStringColumnWithNulls()
|
||||||
|
{
|
||||||
|
// SearchQueryFilter always returns false for null row values.
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue(""), null), ImmutableList.of("1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("10"), null), ImmutableList.of("1"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("2"), null), ImmutableList.of("2"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("1"), null), ImmutableList.of("1", "3"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("def"), null), ImmutableList.of("4"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("abc"), null), ImmutableList.of("5"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("ab"), null), ImmutableList.<String>of("4", "5"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMultiValueStringColumn()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim2", specForValue(""), null), ImmutableList.of("0", "3", "4"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim2", specForValue("a"), null), ImmutableList.of("0", "3"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim2", specForValue("b"), null), ImmutableList.of("0"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim2", specForValue("c"), null), ImmutableList.of("4"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim2", specForValue("d"), null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMissingColumnSpecifiedInDimensionList()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim3", specForValue(""), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim3", specForValue("a"), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim3", specForValue("b"), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim3", specForValue("c"), null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMissingColumnNotSpecifiedInDimensionList()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue(""), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue("a"), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue("b"), null), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue("c"), null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchQueryWithExtractionFn()
|
||||||
|
{
|
||||||
|
String nullJsFn = "function(str) { if (str === null) { return 'NOT_NULL_ANYMORE'; } else { return str;} }";
|
||||||
|
ExtractionFn changeNullFn = new JavaScriptExtractionFn(nullJsFn, false);
|
||||||
|
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("0"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("ab"), changeNullFn), ImmutableList.<String>of("4", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim2", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("1", "2", "5"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim2", specForValue("a"), changeNullFn), ImmutableList.of("0", "3"));
|
||||||
|
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim3", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim3", specForValue("a"), changeNullFn), ImmutableList.<String>of());
|
||||||
|
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue("a"), changeNullFn), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertFilterMatches(
|
||||||
|
final DimFilter filter,
|
||||||
|
final List<String> expectedRows
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||||
|
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||||
|
}
|
||||||
|
}
|
|
@ -29,8 +29,14 @@ import io.druid.data.input.impl.InputRowParser;
|
||||||
import io.druid.data.input.impl.MapInputRowParser;
|
import io.druid.data.input.impl.MapInputRowParser;
|
||||||
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||||
import io.druid.data.input.impl.TimestampSpec;
|
import io.druid.data.input.impl.TimestampSpec;
|
||||||
|
import io.druid.query.extraction.MapLookupExtractor;
|
||||||
import io.druid.query.filter.DimFilter;
|
import io.druid.query.filter.DimFilter;
|
||||||
|
import io.druid.query.filter.ExtractionDimFilter;
|
||||||
|
import io.druid.query.filter.InDimFilter;
|
||||||
|
import io.druid.query.filter.OrDimFilter;
|
||||||
import io.druid.query.filter.SelectorDimFilter;
|
import io.druid.query.filter.SelectorDimFilter;
|
||||||
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
|
import io.druid.query.lookup.LookupExtractor;
|
||||||
import io.druid.segment.IndexBuilder;
|
import io.druid.segment.IndexBuilder;
|
||||||
import io.druid.segment.StorageAdapter;
|
import io.druid.segment.StorageAdapter;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -40,6 +46,7 @@ import org.junit.runner.RunWith;
|
||||||
import org.junit.runners.Parameterized;
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -81,54 +88,146 @@ public class SelectorFilterTest extends BaseFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testSingleValueStringColumnWithoutNulls()
|
public void testSingleValueStringColumnWithoutNulls()
|
||||||
{
|
{
|
||||||
assertFilterMatches(new SelectorDimFilter("dim0", null), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim0", null, null), ImmutableList.<String>of());
|
||||||
assertFilterMatches(new SelectorDimFilter("dim0", ""), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim0", "", null), ImmutableList.<String>of());
|
||||||
assertFilterMatches(new SelectorDimFilter("dim0", "0"), ImmutableList.of("0"));
|
assertFilterMatches(new SelectorDimFilter("dim0", "0", null), ImmutableList.of("0"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim0", "1"), ImmutableList.of("1"));
|
assertFilterMatches(new SelectorDimFilter("dim0", "1", null), ImmutableList.of("1"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSingleValueStringColumnWithNulls()
|
public void testSingleValueStringColumnWithNulls()
|
||||||
{
|
{
|
||||||
assertFilterMatches(new SelectorDimFilter("dim1", null), ImmutableList.of("0"));
|
assertFilterMatches(new SelectorDimFilter("dim1", null, null), ImmutableList.of("0"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim1", ""), ImmutableList.of("0"));
|
assertFilterMatches(new SelectorDimFilter("dim1", "", null), ImmutableList.of("0"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim1", "10"), ImmutableList.of("1"));
|
assertFilterMatches(new SelectorDimFilter("dim1", "10", null), ImmutableList.of("1"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim1", "2"), ImmutableList.of("2"));
|
assertFilterMatches(new SelectorDimFilter("dim1", "2", null), ImmutableList.of("2"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim1", "1"), ImmutableList.of("3"));
|
assertFilterMatches(new SelectorDimFilter("dim1", "1", null), ImmutableList.of("3"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim1", "def"), ImmutableList.of("4"));
|
assertFilterMatches(new SelectorDimFilter("dim1", "def", null), ImmutableList.of("4"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim1", "abc"), ImmutableList.of("5"));
|
assertFilterMatches(new SelectorDimFilter("dim1", "abc", null), ImmutableList.of("5"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim1", "ab"), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim1", "ab", null), ImmutableList.<String>of());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMultiValueStringColumn()
|
public void testMultiValueStringColumn()
|
||||||
{
|
{
|
||||||
assertFilterMatches(new SelectorDimFilter("dim2", null), ImmutableList.of("1", "2", "5"));
|
assertFilterMatches(new SelectorDimFilter("dim2", null, null), ImmutableList.of("1", "2", "5"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim2", ""), ImmutableList.of("1", "2", "5"));
|
assertFilterMatches(new SelectorDimFilter("dim2", "", null), ImmutableList.of("1", "2", "5"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim2", "a"), ImmutableList.of("0", "3"));
|
assertFilterMatches(new SelectorDimFilter("dim2", "a", null), ImmutableList.of("0", "3"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim2", "b"), ImmutableList.of("0"));
|
assertFilterMatches(new SelectorDimFilter("dim2", "b", null), ImmutableList.of("0"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim2", "c"), ImmutableList.of("4"));
|
assertFilterMatches(new SelectorDimFilter("dim2", "c", null), ImmutableList.of("4"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim2", "d"), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim2", "d", null), ImmutableList.<String>of());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMissingColumnSpecifiedInDimensionList()
|
public void testMissingColumnSpecifiedInDimensionList()
|
||||||
{
|
{
|
||||||
assertFilterMatches(new SelectorDimFilter("dim3", null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
assertFilterMatches(new SelectorDimFilter("dim3", null, null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim3", ""), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
assertFilterMatches(new SelectorDimFilter("dim3", "", null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim3", "a"), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim3", "a", null), ImmutableList.<String>of());
|
||||||
assertFilterMatches(new SelectorDimFilter("dim3", "b"), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim3", "b", null), ImmutableList.<String>of());
|
||||||
assertFilterMatches(new SelectorDimFilter("dim3", "c"), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim3", "c", null), ImmutableList.<String>of());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMissingColumnNotSpecifiedInDimensionList()
|
public void testMissingColumnNotSpecifiedInDimensionList()
|
||||||
{
|
{
|
||||||
assertFilterMatches(new SelectorDimFilter("dim4", null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
assertFilterMatches(new SelectorDimFilter("dim4", null, null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim4", ""), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
assertFilterMatches(new SelectorDimFilter("dim4", "", null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
assertFilterMatches(new SelectorDimFilter("dim4", "a"), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim4", "a", null), ImmutableList.<String>of());
|
||||||
assertFilterMatches(new SelectorDimFilter("dim4", "b"), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim4", "b", null), ImmutableList.<String>of());
|
||||||
assertFilterMatches(new SelectorDimFilter("dim4", "c"), ImmutableList.<String>of());
|
assertFilterMatches(new SelectorDimFilter("dim4", "c", null), ImmutableList.<String>of());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSelectorWithLookupExtractionFn()
|
||||||
|
{
|
||||||
|
final Map<String, String> stringMap = ImmutableMap.of(
|
||||||
|
"1", "HELLO",
|
||||||
|
"a", "HELLO",
|
||||||
|
"def", "HELLO",
|
||||||
|
"abc", "UNKNOWN"
|
||||||
|
);
|
||||||
|
LookupExtractor mapExtractor = new MapLookupExtractor(stringMap, false);
|
||||||
|
LookupExtractionFn lookupFn = new LookupExtractionFn(mapExtractor, false, "UNKNOWN", false, true);
|
||||||
|
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim0", "HELLO", lookupFn), ImmutableList.of("1"));
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim0", "UNKNOWN", lookupFn), ImmutableList.of("0", "2", "3", "4", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim1", "HELLO", lookupFn), ImmutableList.of("3", "4"));
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim1", "UNKNOWN", lookupFn), ImmutableList.of("0", "1", "2", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim2", "HELLO", lookupFn), ImmutableList.of("0", "3"));
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim2", "UNKNOWN", lookupFn), ImmutableList.of("0", "1", "2", "4", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim3", "HELLO", lookupFn), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim3", "UNKNOWN", lookupFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim4", "HELLO", lookupFn), ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim4", "UNKNOWN", lookupFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
|
||||||
|
final Map<String, String> stringMap2 = ImmutableMap.of(
|
||||||
|
"2", "5"
|
||||||
|
);
|
||||||
|
LookupExtractor mapExtractor2 = new MapLookupExtractor(stringMap2, false);
|
||||||
|
LookupExtractionFn lookupFn2 = new LookupExtractionFn(mapExtractor2, true, null, false, true);
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim0", "5", lookupFn2), ImmutableList.of("2", "5"));
|
||||||
|
|
||||||
|
final Map<String, String> stringMap3 = ImmutableMap.of(
|
||||||
|
"1", ""
|
||||||
|
);
|
||||||
|
LookupExtractor mapExtractor3 = new MapLookupExtractor(stringMap3, false);
|
||||||
|
LookupExtractionFn lookupFn3 = new LookupExtractionFn(mapExtractor3, false, null, false, true);
|
||||||
|
assertFilterMatches(new SelectorDimFilter("dim0", null, lookupFn3), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
|
||||||
|
final Map<String, String> stringMap4 = ImmutableMap.of(
|
||||||
|
"9", "4"
|
||||||
|
);
|
||||||
|
LookupExtractor mapExtractor4 = new MapLookupExtractor(stringMap4, false);
|
||||||
|
LookupExtractionFn lookupFn4 = new LookupExtractionFn(mapExtractor4, true, null, false, true);
|
||||||
|
|
||||||
|
final Map<String, String> stringMap5 = ImmutableMap.of(
|
||||||
|
"5", "44"
|
||||||
|
);
|
||||||
|
LookupExtractor mapExtractor5 = new MapLookupExtractor(stringMap5, false);
|
||||||
|
LookupExtractionFn lookupFn5 = new LookupExtractionFn(mapExtractor5, true, null, false, true);
|
||||||
|
|
||||||
|
final Map<String, String> stringMap6 = ImmutableMap.of(
|
||||||
|
"5", "5"
|
||||||
|
);
|
||||||
|
LookupExtractor mapExtractor6 = new MapLookupExtractor(stringMap6, false);
|
||||||
|
LookupExtractionFn lookupFn6 = new LookupExtractionFn(mapExtractor6, true, null, false, true);
|
||||||
|
|
||||||
|
// optimize() tests, check that filter was converted to the proper form
|
||||||
|
SelectorDimFilter optFilter1 = new SelectorDimFilter("dim1", "UNKNOWN", lookupFn);
|
||||||
|
SelectorDimFilter optFilter2 = new SelectorDimFilter("dim0", "5", lookupFn2);
|
||||||
|
SelectorDimFilter optFilter3 = new SelectorDimFilter("dim0", null, lookupFn3);
|
||||||
|
SelectorDimFilter optFilter4 = new SelectorDimFilter("dim0", "5", lookupFn4);
|
||||||
|
SelectorDimFilter optFilter5 = new SelectorDimFilter("dim0", "5", lookupFn5);
|
||||||
|
SelectorDimFilter optFilter6 = new SelectorDimFilter("dim0", "5", lookupFn6);
|
||||||
|
|
||||||
|
InDimFilter optFilter2Optimized = new InDimFilter("dim0", Arrays.asList("2", "5"), null);
|
||||||
|
SelectorDimFilter optFilter4Optimized = new SelectorDimFilter("dim0", "5", null);
|
||||||
|
SelectorDimFilter optFilter6Optimized = new SelectorDimFilter("dim0", "5", null);
|
||||||
|
|
||||||
|
Assert.assertTrue(optFilter1 == optFilter1.optimize());
|
||||||
|
Assert.assertTrue(optFilter2Optimized.equals(optFilter2.optimize()));
|
||||||
|
Assert.assertTrue(optFilter3 == optFilter3.optimize());
|
||||||
|
Assert.assertTrue(optFilter4Optimized.equals(optFilter4.optimize()));
|
||||||
|
Assert.assertTrue(optFilter5 == optFilter5.optimize());
|
||||||
|
Assert.assertTrue(optFilter6Optimized.equals(optFilter6.optimize()));
|
||||||
|
|
||||||
|
assertFilterMatches(optFilter1, ImmutableList.of("0", "1", "2", "5"));
|
||||||
|
assertFilterMatches(optFilter2, ImmutableList.of("2", "5"));
|
||||||
|
assertFilterMatches(optFilter3, ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
|
assertFilterMatches(optFilter4, ImmutableList.of("5"));
|
||||||
|
assertFilterMatches(optFilter5, ImmutableList.<String>of());
|
||||||
|
assertFilterMatches(optFilter6, ImmutableList.of("5"));
|
||||||
|
|
||||||
|
// tests that ExtractionDimFilter (identical to SelectorDimFilter now) optimize() with lookup works
|
||||||
|
// remove these when ExtractionDimFilter is removed.
|
||||||
|
assertFilterMatches(new ExtractionDimFilter("dim1", "UNKNOWN", lookupFn, null), ImmutableList.of("0", "1", "2", "5"));
|
||||||
|
assertFilterMatches(new ExtractionDimFilter("dim0", "5", lookupFn2, null), ImmutableList.of("2", "5"));
|
||||||
|
assertFilterMatches(new ExtractionDimFilter("dim0", null, lookupFn3, null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertFilterMatches(
|
private void assertFilterMatches(
|
||||||
|
|
|
@ -259,7 +259,7 @@ public class IncrementalIndexStorageAdapterTest
|
||||||
|
|
||||||
for (boolean descending : Arrays.asList(false, true)) {
|
for (boolean descending : Arrays.asList(false, true)) {
|
||||||
Sequence<Cursor> cursorSequence = adapter.makeCursors(
|
Sequence<Cursor> cursorSequence = adapter.makeCursors(
|
||||||
new SelectorFilter("sally", "bo"),
|
new SelectorFilter("sally", "bo", null),
|
||||||
interval,
|
interval,
|
||||||
QueryGranularity.NONE,
|
QueryGranularity.NONE,
|
||||||
descending
|
descending
|
||||||
|
|
|
@ -82,7 +82,7 @@ public class IncrementalIndexTest
|
||||||
AggregatorFactory[] metrics = {
|
AggregatorFactory[] metrics = {
|
||||||
new FilteredAggregatorFactory(
|
new FilteredAggregatorFactory(
|
||||||
new CountAggregatorFactory("cnt"),
|
new CountAggregatorFactory("cnt"),
|
||||||
new SelectorDimFilter("billy", "A")
|
new SelectorDimFilter("billy", "A", null)
|
||||||
)
|
)
|
||||||
};
|
};
|
||||||
final IncrementalIndexSchema schema = new IncrementalIndexSchema(
|
final IncrementalIndexSchema schema = new IncrementalIndexSchema(
|
||||||
|
|
|
@ -297,9 +297,9 @@ selectorDimFilter returns [DimFilter filter]
|
||||||
String dim = $dimension.text;
|
String dim = $dimension.text;
|
||||||
String val = unescape($value.text);
|
String val = unescape($value.text);
|
||||||
switch($op.type) {
|
switch($op.type) {
|
||||||
case(EQ): $filter = new SelectorDimFilter(dim, val); break;
|
case(EQ): $filter = new SelectorDimFilter(dim, val, null); break;
|
||||||
case(NEQ): $filter = new NotDimFilter(new SelectorDimFilter(dim, val)); break;
|
case(NEQ): $filter = new NotDimFilter(new SelectorDimFilter(dim, val, null)); break;
|
||||||
case(MATCH): $filter = new RegexDimFilter(dim, val); break;
|
case(MATCH): $filter = new RegexDimFilter(dim, val, null); break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
;
|
;
|
||||||
|
@ -307,7 +307,7 @@ selectorDimFilter returns [DimFilter filter]
|
||||||
inListDimFilter returns [DimFilter filter]
|
inListDimFilter returns [DimFilter filter]
|
||||||
: dimension=IDENT 'in' (OPEN! ( (list+=QUOTED_STRING (COMMA! list+=QUOTED_STRING)*) ) CLOSE!) {
|
: dimension=IDENT 'in' (OPEN! ( (list+=QUOTED_STRING (COMMA! list+=QUOTED_STRING)*) ) CLOSE!) {
|
||||||
List<DimFilter> filterList = new LinkedList<DimFilter>();
|
List<DimFilter> filterList = new LinkedList<DimFilter>();
|
||||||
for(Token e : $list) filterList.add(new SelectorDimFilter($dimension.text, unescape(e.getText())));
|
for(Token e : $list) filterList.add(new SelectorDimFilter($dimension.text, unescape(e.getText()), null));
|
||||||
$filter = new OrDimFilter(filterList);
|
$filter = new OrDimFilter(filterList);
|
||||||
}
|
}
|
||||||
;
|
;
|
||||||
|
|
Loading…
Reference in New Issue