Add time interval dim filter and retention analysis example (#3315)

* Add time interval dim filter and retention analysis example

* Use closed-open matching for intervals, update cache key generation

* Fix time filtering tests for interval boundary change
This commit is contained in:
Jonathan Wei 2016-08-05 07:25:04 -07:00 committed by Gian Merlino
parent 1e3979a5e8
commit decefb7477
11 changed files with 965 additions and 23 deletions

View File

@ -153,3 +153,101 @@ sample query for, How many unique users visited both product A and B?
]
}
```
#### Retention Analysis Example
Suppose you want to answer a question like, "How many unique users performed a specific action in a particular time period and also performed another specific action in a different time period?"
e.g., "How many unique users signed up in week 1, and purchased something in week 2?"
Using the `(timestamp, product, user_id)` example dataset, data would be indexed with the following aggregator, like in the example above:
```json
{ "type": "thetaSketch", "name": "user_id_sketch", "fieldName": "user_id" }
```
The following query expresses:
"Out of the unique users who visited Product A between 10/01/2014 and 10/07/2014, how many visited Product A again in the week of 10/08/2014 to 10/14/2014?"
```json
{
"queryType": "groupBy",
"dataSource": "test_datasource",
"granularity": "ALL",
"dimensions": [],
"filter": {
"type": "or",
"fields": [
{"type": "selector", "dimension": "product", "value": "A"}
]
},
"aggregations": [
{
"type" : "filtered",
"filter" : {
"type" : "and",
"fields" : [
{
"type" : "selector",
"dimension" : "product",
"value" : "A"
},
{
"type" : "interval",
"dimension" : "__time",
"intervals" : ["2014-10-01T00:00:00.000Z/2014-10-07T00:00:00.000Z"]
}
]
},
"aggregator" : {
"type": "thetaSketch", "name": "A_unique_users_week_1", "fieldName": "user_id_sketch"
}
},
{
"type" : "filtered",
"filter" : {
"type" : "and",
"fields" : [
{
"type" : "selector",
"dimension" : "product",
"value" : "A"
},
{
"type" : "interval",
"dimension" : "__time",
"intervals" : ["2014-10-08T00:00:00.000Z/2014-10-14T00:00:00.000Z"]
}
]
},
"aggregator" : {
"type": "thetaSketch", "name": "A_unique_users_week_2", "fieldName": "user_id_sketch"
}
},
],
"postAggregations": [
{
"type": "thetaSketchEstimate",
"name": "final_unique_users",
"field":
{
"type": "thetaSketchSetOp",
"name": "final_unique_users_sketch",
"func": "INTERSECT",
"fields": [
{
"type": "fieldAccess",
"fieldName": "A_unique_users_week_1"
},
{
"type": "fieldAccess",
"fieldName": "A_unique_users_week_2"
}
]
}
}
],
"intervals": ["2014-10-01T00:00:00.000Z/2014-10-14T00:00:00.000Z"]
}
```

View File

@ -150,6 +150,33 @@ Search filters can be used to filter on partial string matches.
The search filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
#### Search Query Spec
##### Contains
|property|description|required?|
|--------|-----------|---------|
|type|This String should always be "contains".|yes|
|value|A String value to run the search over.|yes|
|caseSensitive|Whether two string should be compared as case sensitive or not|no (default == false)|
##### Insensitive Contains
|property|description|required?|
|--------|-----------|---------|
|type|This String should always be "insensitive_contains".|yes|
|value|A String value to run the search over.|yes|
Note that an "insensitive_contains" search is equivalent to a "contains" search with "caseSensitive": false (or not
provided).
##### Fragment
|property|description|required?|
|--------|-----------|---------|
|type|This String should always be "fragment".|yes|
|values|A JSON array of String values to run the search over.|yes|
|caseSensitive|Whether strings should be compared as case sensitive or not. Default: false(insensitive)|no|
### In filter
@ -245,33 +272,62 @@ Likewise, this filter expresses `age >= 18`
```
#### Search Query Spec
### Interval Filter
##### Contains
The Interval filter enables range filtering on columns that contain long millisecond values, with the boundaries specified as ISO 8601 time intervals. It is suitable for the `__time` column, long metric columns, and dimensions with values that can be parsed as long milliseconds.
|property|description|required?|
|--------|-----------|---------|
|type|This String should always be "contains".|yes|
|value|A String value to run the search over.|yes|
|caseSensitive|Whether two string should be compared as case sensitive or not|no (default == false)|
This filter converts the ISO 8601 intervals to long millisecond start/end ranges and translates to an OR of Bound filters on those millisecond ranges, with numeric comparison. The Bound filters will have left-closed and right-open matching (i.e., start <= time < end).
##### Insensitive Contains
|property|type|description|required?|
|--------|-----------|---------|---------|
|type|String|This should always be "interval".|yes|
|dimension|String|The dimension to filter on|yes|
|intervals|Array|A JSON array containing ISO-8601 interval strings. This defines the time ranges to filter on.|yes|
|extractionFn|[Extraction function](#filtering-with-extraction-functions)| Extraction function to apply to the dimension|no|
|property|description|required?|
|--------|-----------|---------|
|type|This String should always be "insensitive_contains".|yes|
|value|A String value to run the search over.|yes|
The interval filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
Note that an "insensitive_contains" search is equivalent to a "contains" search with "caseSensitive": false (or not
provided).
If an extraction function is used with this filter, the extraction function should output values that are parseable as long milliseconds.
##### Fragment
The following example filters on the time ranges of October 1-7, 2014 and November 15-16, 2014.
```json
{
"type" : "interval",
"dimension" : "__time",
"intervals" : [
"2014-10-01T00:00:00.000Z/2014-10-07T00:00:00.000Z",
"2014-11-15T00:00:00.000Z/2014-11-16T00:00:00.000Z"
]
}
```
|property|description|required?|
|--------|-----------|---------|
|type|This String should always be "fragment".|yes|
|values|A JSON array of String values to run the search over.|yes|
|caseSensitive|Whether strings should be compared as case sensitive or not. Default: false(insensitive)|no|
The filter above is equivalent to the following OR of Bound filters:
```json
{
"type": "or",
"fields": [
{
"type": "bound",
"dimension": "__time",
"lower": "1412121600000",
"lowerStrict": false,
"upper": "1412640000000" ,
"upperStrict": true,
"ordering": "numeric"
},
{
"type": "bound",
"dimension": "__time",
"lower": "1416009600000",
"lowerStrict": false,
"upper": "1416096000000" ,
"upperStrict": true,
"ordering": "numeric"
}
]
}
```
### Filtering with Extraction Functions
Some filters optionally support the use of extraction functions.
@ -343,3 +399,15 @@ Filtering on day of week:
}
}
```
Filtering on a set of ISO 8601 intervals:
```json
{
"type" : "interval",
"dimension" : "__time",
"intervals" : [
"2014-10-01T00:00:00.000Z/2014-10-07T00:00:00.000Z",
"2014-11-15T00:00:00.000Z/2014-11-16T00:00:00.000Z"
]
}
```

View File

@ -417,6 +417,42 @@ public class SketchAggregationTest
Assert.assertFalse(Arrays.equals(factory1.getCacheKey(), factory3.getCacheKey()));
}
@Test
public void testRetentionDataIngestAndGpByQuery() throws Exception
{
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
new File(this.getClass().getClassLoader().getResource("retention_test_data.tsv").getFile()),
readFileFromClasspathAsString("simple_test_data_record_parser.json"),
readFileFromClasspathAsString("simple_test_data_aggregators.json"),
0,
QueryGranularities.NONE,
5,
readFileFromClasspathAsString("retention_test_data_group_by_query.json")
);
List<Row> results = Sequences.toList(seq, Lists.<Row>newArrayList());
Assert.assertEquals(1, results.size());
Assert.assertEquals(
ImmutableList.of(
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_1")
.put("p1_unique_country_day_1", 20.0)
.put("p1_unique_country_day_2", 20.0)
.put("p1_unique_country_day_3", 10.0)
.put("sketchEstimatePostAgg", 20.0)
.put("sketchIntersectionPostAggEstimate1", 10.0)
.put("sketchIntersectionPostAggEstimate2", 5.0)
.put("non_existing_col_validation", 0.0)
.build()
)
),
results
);
}
private void assertPostAggregatorSerde(PostAggregator agg) throws Exception
{
Assert.assertEquals(

View File

@ -0,0 +1,50 @@
2014102001 product_1 pty_country_1
2014102001 product_1 pty_country_2
2014102001 product_1 pty_country_3
2014102001 product_1 pty_country_4
2014102001 product_1 pty_country_5
2014102001 product_1 pty_country_6
2014102001 product_1 pty_country_7
2014102001 product_1 pty_country_8
2014102001 product_1 pty_country_9
2014102001 product_1 pty_country_10
2014102001 product_1 pty_country_11
2014102001 product_1 pty_country_12
2014102001 product_1 pty_country_13
2014102001 product_1 pty_country_14
2014102001 product_1 pty_country_15
2014102001 product_1 pty_country_16
2014102001 product_1 pty_country_17
2014102001 product_1 pty_country_18
2014102001 product_1 pty_country_19
2014102001 product_1 pty_country_20
2014102101 product_1 pty_country_1
2014102101 product_1 pty_country_2
2014102101 product_1 pty_country_3
2014102101 product_1 pty_country_4
2014102101 product_1 pty_country_5
2014102101 product_1 pty_country_6
2014102101 product_1 pty_country_7
2014102101 product_1 pty_country_8
2014102101 product_1 pty_country_9
2014102101 product_1 pty_country_10
2014102101 product_1 pty_country_50
2014102101 product_1 pty_country_51
2014102101 product_1 pty_country_52
2014102101 product_1 pty_country_53
2014102101 product_1 pty_country_54
2014102101 product_1 pty_country_55
2014102101 product_1 pty_country_56
2014102101 product_1 pty_country_57
2014102101 product_1 pty_country_58
2014102101 product_1 pty_country_59
2014102201 product_1 pty_country_1
2014102201 product_1 pty_country_2
2014102201 product_1 pty_country_3
2014102201 product_1 pty_country_4
2014102201 product_1 pty_country_5
2014102201 product_1 pty_country_60
2014102201 product_1 pty_country_61
2014102201 product_1 pty_country_62
2014102201 product_1 pty_country_63
2014102201 product_1 pty_country_64
1 2014102001 product_1 pty_country_1
2 2014102001 product_1 pty_country_2
3 2014102001 product_1 pty_country_3
4 2014102001 product_1 pty_country_4
5 2014102001 product_1 pty_country_5
6 2014102001 product_1 pty_country_6
7 2014102001 product_1 pty_country_7
8 2014102001 product_1 pty_country_8
9 2014102001 product_1 pty_country_9
10 2014102001 product_1 pty_country_10
11 2014102001 product_1 pty_country_11
12 2014102001 product_1 pty_country_12
13 2014102001 product_1 pty_country_13
14 2014102001 product_1 pty_country_14
15 2014102001 product_1 pty_country_15
16 2014102001 product_1 pty_country_16
17 2014102001 product_1 pty_country_17
18 2014102001 product_1 pty_country_18
19 2014102001 product_1 pty_country_19
20 2014102001 product_1 pty_country_20
21 2014102101 product_1 pty_country_1
22 2014102101 product_1 pty_country_2
23 2014102101 product_1 pty_country_3
24 2014102101 product_1 pty_country_4
25 2014102101 product_1 pty_country_5
26 2014102101 product_1 pty_country_6
27 2014102101 product_1 pty_country_7
28 2014102101 product_1 pty_country_8
29 2014102101 product_1 pty_country_9
30 2014102101 product_1 pty_country_10
31 2014102101 product_1 pty_country_50
32 2014102101 product_1 pty_country_51
33 2014102101 product_1 pty_country_52
34 2014102101 product_1 pty_country_53
35 2014102101 product_1 pty_country_54
36 2014102101 product_1 pty_country_55
37 2014102101 product_1 pty_country_56
38 2014102101 product_1 pty_country_57
39 2014102101 product_1 pty_country_58
40 2014102101 product_1 pty_country_59
41 2014102201 product_1 pty_country_1
42 2014102201 product_1 pty_country_2
43 2014102201 product_1 pty_country_3
44 2014102201 product_1 pty_country_4
45 2014102201 product_1 pty_country_5
46 2014102201 product_1 pty_country_60
47 2014102201 product_1 pty_country_61
48 2014102201 product_1 pty_country_62
49 2014102201 product_1 pty_country_63
50 2014102201 product_1 pty_country_64

View File

@ -0,0 +1,135 @@
{
"queryType": "groupBy",
"dataSource": "test_datasource",
"granularity":"ALL",
"dimensions": ["product"],
"filter": {
"type": "selector",
"dimension": "product",
"value": "product_1"
},
"aggregations": [
{
"type" : "filtered",
"filter" : {
"type" : "and",
"fields" : [
{
"type" : "selector",
"dimension" : "product",
"value" : "product_1"
},
{
"type" : "interval",
"dimension" : "__time",
"intervals" : ["2014-10-20T00:00:00.000Z/2014-10-21T00:00:00.000Z"]
}
]
},
"aggregator" : {
"type": "thetaSketch", "name": "p1_unique_country_day_1", "fieldName": "pty_country"
}
},
{
"type" : "filtered",
"filter" : {
"type" : "and",
"fields" : [
{
"type" : "selector",
"dimension" : "product",
"value" : "product_1"
},
{
"type" : "interval",
"dimension" : "__time",
"intervals" : ["2014-10-21T00:00:00.000Z/2014-10-22T00:00:00.000Z"]
}
]
},
"aggregator" : {
"type": "thetaSketch", "name": "p1_unique_country_day_2", "fieldName": "pty_country"
}
},
{
"type" : "filtered",
"filter" : {
"type" : "and",
"fields" : [
{
"type" : "selector",
"dimension" : "product",
"value" : "product_1"
},
{
"type" : "interval",
"dimension" : "__time",
"intervals" : ["2014-10-22T00:00:00.000Z/2014-10-23T00:00:00.000Z"]
}
]
},
"aggregator" : {
"type": "thetaSketch", "name": "p1_unique_country_day_3", "fieldName": "pty_country"
}
},
{
"type": "thetaSketch",
"name": "non_existing_col_validation",
"fieldName": "non_existing_col",
"size": 16384
}
],
"postAggregations": [
{
"type": "thetaSketchEstimate",
"name": "sketchEstimatePostAgg",
"field": {
"type": "fieldAccess",
"fieldName": "p1_unique_country_day_1"
}
},
{
"type": "thetaSketchEstimate",
"name": "sketchIntersectionPostAggEstimate1",
"field": {
"type": "thetaSketchSetOp",
"name": "sketchIntersectionPostAgg",
"func": "INTERSECT",
"size": 16384,
"fields": [
{
"type": "fieldAccess",
"fieldName": "p1_unique_country_day_1"
},
{
"type": "fieldAccess",
"fieldName": "p1_unique_country_day_2"
}
]
}
},
{
"type": "thetaSketchEstimate",
"name": "sketchIntersectionPostAggEstimate2",
"field": {
"type": "thetaSketchSetOp",
"name": "sketchIntersectionPostAgg2",
"func": "INTERSECT",
"size": 16384,
"fields": [
{
"type": "fieldAccess",
"fieldName": "p1_unique_country_day_1"
},
{
"type": "fieldAccess",
"fieldName": "p1_unique_country_day_3"
}
]
}
}
],
"intervals": [
"2014-10-19T00:00:00.000Z/2014-10-23T00:00:00.000Z"
]
}

View File

@ -37,8 +37,8 @@ import com.google.common.collect.RangeSet;
@JsonSubTypes.Type(name="javascript", value=JavaScriptDimFilter.class),
@JsonSubTypes.Type(name="spatial", value=SpatialDimFilter.class),
@JsonSubTypes.Type(name="in", value=InDimFilter.class),
@JsonSubTypes.Type(name="bound", value=BoundDimFilter.class)
@JsonSubTypes.Type(name="bound", value=BoundDimFilter.class),
@JsonSubTypes.Type(name="interval", value=IntervalDimFilter.class)
})
public interface DimFilter
{

View File

@ -47,8 +47,9 @@ public class DimFilterUtils
static final byte JAVASCRIPT_CACHE_ID = 0x7;
static final byte SPATIAL_CACHE_ID = 0x8;
static final byte IN_CACHE_ID = 0x9;
static final byte BOUND_CACHE_ID = 0xA;
static final byte INTERVAL_CACHE_ID = 0xB;
public static final byte STRING_SEPARATOR = (byte) 0xFF;
public static byte BOUND_CACHE_ID = 0xA;
static byte[] computeCacheKey(byte cacheIdKey, List<DimFilter> filters)
{

View File

@ -0,0 +1,186 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.filter;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.RangeSet;
import com.google.common.primitives.Longs;
import com.metamx.common.Pair;
import com.metamx.common.StringUtils;
import io.druid.common.utils.JodaUtils;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.filter.OrFilter;
import org.joda.time.Interval;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
public class IntervalDimFilter implements DimFilter
{
private final List<Interval> intervals;
private final List<Pair<Long, Long>> intervalLongs;
private final String dimension;
private final ExtractionFn extractionFn;
private final OrDimFilter convertedFilter;
@JsonCreator
public IntervalDimFilter(
@JsonProperty("dimension") String dimension,
@JsonProperty("intervals") List<Interval> intervals,
@JsonProperty("extractionFn") ExtractionFn extractionFn
)
{
Preconditions.checkNotNull(dimension, "dimension can not be null");
Preconditions.checkNotNull(intervals, "intervals can not be null");
Preconditions.checkArgument(intervals.size() > 0, "must specify at least one interval");
this.dimension = dimension;
this.intervals = Collections.unmodifiableList(JodaUtils.condenseIntervals(intervals));
this.extractionFn = extractionFn;
this.intervalLongs = makeIntervalLongs();
this.convertedFilter = new OrDimFilter(makeBoundDimFilters());
}
@JsonProperty
public String getDimension()
{
return dimension;
}
@JsonProperty
public List<Interval> getIntervals()
{
return intervals;
}
@JsonProperty
public ExtractionFn getExtractionFn()
{
return extractionFn;
}
@Override
public byte[] getCacheKey()
{
byte[] dimensionBytes = StringUtils.toUtf8(dimension);
byte[] extractionFnBytes = extractionFn == null ? new byte[0] : extractionFn.getCacheKey();
int intervalsBytesSize = intervalLongs.size() * Longs.BYTES * 2 + intervalLongs.size();
ByteBuffer filterCacheKey = ByteBuffer.allocate(3
+ dimensionBytes.length
+ intervalsBytesSize
+ extractionFnBytes.length)
.put(DimFilterUtils.INTERVAL_CACHE_ID)
.put(dimensionBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(extractionFnBytes)
.put(DimFilterUtils.STRING_SEPARATOR);
for (Pair<Long, Long> interval : intervalLongs) {
filterCacheKey.put(Longs.toByteArray(interval.lhs))
.put(Longs.toByteArray(interval.rhs))
.put((byte) 0xFF);
}
return filterCacheKey.array();
}
@Override
public DimFilter optimize()
{
return this;
}
@Override
public Filter toFilter()
{
return convertedFilter.toFilter();
}
@Override
public RangeSet<String> getDimensionRangeSet(String dimension)
{
return convertedFilter.getDimensionRangeSet(dimension);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
IntervalDimFilter that = (IntervalDimFilter) o;
if (!getIntervals().equals(that.getIntervals())) {
return false;
}
if (!getDimension().equals(that.getDimension())) {
return false;
}
return getExtractionFn() != null
? getExtractionFn().equals(that.getExtractionFn())
: that.getExtractionFn() == null;
}
@Override
public int hashCode()
{
int result = getIntervals().hashCode();
result = 31 * result + getDimension().hashCode();
result = 31 * result + (getExtractionFn() != null ? getExtractionFn().hashCode() : 0);
return result;
}
private List<Pair<Long, Long>> makeIntervalLongs()
{
List<Pair<Long, Long>> intervalLongs = new ArrayList<>();
for (Interval interval : intervals) {
intervalLongs.add(new Pair<Long, Long>(interval.getStartMillis(), interval.getEndMillis()));
}
return intervalLongs;
}
private List<DimFilter> makeBoundDimFilters()
{
List<DimFilter> boundDimFilters = new ArrayList<>();
for (Pair<Long, Long> interval : intervalLongs) {
BoundDimFilter boundDimFilter = new BoundDimFilter(
dimension,
String.valueOf(interval.lhs),
String.valueOf(interval.rhs),
false,
true,
null,
extractionFn,
StringComparators.NUMERIC
);
boundDimFilters.add(boundDimFilter);
}
return boundDimFilters;
}
}

View File

@ -27,6 +27,8 @@ import io.druid.js.JavaScriptConfig;
import io.druid.query.extraction.IdentityExtractionFn;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import io.druid.segment.column.Column;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
@ -61,6 +63,24 @@ public class GetDimensionRangeSetTest
JavaScriptConfig.getDefault());
private final DimFilter other3 = new SearchQueryDimFilter("dim", new ContainsSearchQuerySpec("a", true), null);
private final DimFilter interval1 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
private final DimFilter interval2 = new IntervalDimFilter(
"dim1",
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
private static final RangeSet all = rangeSet(ImmutableList.of(Range.<String>all()));
private static final RangeSet empty = rangeSet(ImmutableList.<Range<String>>of());
@ -91,6 +111,9 @@ public class GetDimensionRangeSetTest
Assert.assertNull(other1.getDimensionRangeSet("someDim"));
Assert.assertNull(other2.getDimensionRangeSet("someOtherDim"));
Assert.assertNull(other3.getDimensionRangeSet("dim"));
Assert.assertNull(interval1.getDimensionRangeSet(Column.TIME_COLUMN_NAME));
Assert.assertNull(interval2.getDimensionRangeSet("dim1"));
}
@Test

View File

@ -0,0 +1,233 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.filter;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Injector;
import com.google.inject.Key;
import io.druid.guice.GuiceInjectors;
import io.druid.guice.annotations.Json;
import io.druid.query.extraction.RegexDimExtractionFn;
import io.druid.segment.column.Column;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
public class IntervalDimFilterTest
{
private static ObjectMapper mapper;
@Before
public void setUp()
{
Injector defaultInjector = GuiceInjectors.makeStartupInjector();
mapper = defaultInjector.getInstance(Key.get(ObjectMapper.class, Json.class));
}
@Test
public void testSerde() throws IOException
{
DimFilter intervalFilter = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
String filterStr = mapper.writeValueAsString(intervalFilter);
IntervalDimFilter actualFilter = mapper.reader(DimFilter.class).readValue(filterStr);
Assert.assertEquals(intervalFilter, actualFilter);
intervalFilter = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
new RegexDimExtractionFn(".*", false, null)
);
filterStr = mapper.writeValueAsString(intervalFilter);
actualFilter = mapper.reader(DimFilter.class).readValue(filterStr);
Assert.assertEquals(intervalFilter, actualFilter);
}
@Test
public void testGetCacheKey()
{
DimFilter intervalFilter1 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
DimFilter intervalFilter2 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1976-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
Assert.assertNotEquals(intervalFilter1.getCacheKey(), intervalFilter2.getCacheKey());
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
DimFilter intervalFilter3 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
regexFn
);
DimFilter intervalFilter4 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1976-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
regexFn
);
Assert.assertNotEquals(intervalFilter3.getCacheKey(), intervalFilter4.getCacheKey());
}
@Test
public void testHashCode()
{
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
DimFilter intervalFilter1 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
DimFilter intervalFilter2 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
regexFn
);
DimFilter intervalFilter3 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1977-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
Assert.assertNotEquals(intervalFilter1.hashCode(), intervalFilter2.hashCode());
Assert.assertNotEquals(intervalFilter1.hashCode(), intervalFilter3.hashCode());
DimFilter intervalFilter4 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1977-01-01T00:00:00.004Z"),
Interval.parse("1976-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
Assert.assertEquals(intervalFilter1.hashCode(), intervalFilter4.hashCode());
DimFilter intervalFilter5 = new IntervalDimFilter(
"__thyme",
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
Assert.assertNotEquals(intervalFilter1.hashCode(), intervalFilter5.hashCode());
}
@Test
public void testEquals()
{
RegexDimExtractionFn regexFn = new RegexDimExtractionFn(".*", false, null);
DimFilter intervalFilter1 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
DimFilter intervalFilter2 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
regexFn
);
DimFilter intervalFilter3 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1977-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
Assert.assertNotEquals(intervalFilter1, intervalFilter2);
Assert.assertNotEquals(intervalFilter1, intervalFilter3);
DimFilter intervalFilter4 = new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1977-01-01T00:00:00.004Z"),
Interval.parse("1976-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
Assert.assertEquals(intervalFilter1, intervalFilter4);
DimFilter intervalFilter5 = new IntervalDimFilter(
"__thyme",
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.004Z"),
Interval.parse("1975-01-01T00:00:00.001Z/1980-01-01T00:00:00.004Z")
),
null
);
Assert.assertNotEquals(intervalFilter1, intervalFilter5);
}
}

View File

@ -31,11 +31,13 @@ import io.druid.data.input.impl.TimeAndDimsParseSpec;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.js.JavaScriptConfig;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.JavaScriptExtractionFn;
import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.extraction.TimeFormatExtractionFn;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.InDimFilter;
import io.druid.query.filter.IntervalDimFilter;
import io.druid.query.filter.JavaScriptDimFilter;
import io.druid.query.filter.RegexDimFilter;
import io.druid.query.filter.SearchQueryDimFilter;
@ -49,6 +51,7 @@ import io.druid.segment.StorageAdapter;
import io.druid.segment.column.Column;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Test;
@ -230,6 +233,115 @@ public class TimeFilteringTest extends BaseFilterTest
);
}
@Test
public void testIntervalFilter()
{
assertFilterMatches(
new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.005Z")),
null
),
ImmutableList.<String>of("1", "2", "3", "4")
);
assertFilterMatches(
new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.000Z/1970-01-01T00:00:00.003Z"),
Interval.parse("1970-01-01T00:00:00.004Z/1970-01-01T00:00:00.006Z")
),
null
),
ImmutableList.<String>of("0", "1", "2", "4", "5")
);
assertFilterMatches(
new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.000Z/1970-01-01T00:00:00.001Z"),
Interval.parse("1970-01-01T00:00:00.003Z/1970-01-01T00:00:00.006Z"),
Interval.parse("1970-01-01T00:00:00.002Z/1970-01-01T00:00:00.005Z")
),
null
),
ImmutableList.<String>of("0", "2", "3", "4", "5")
);
// increment timestamp by 2 hours
String timeBoosterJsFn = "function(x) { return(x + 7200000) }";
ExtractionFn exFn = new JavaScriptExtractionFn(timeBoosterJsFn, true, JavaScriptConfig.getDefault());
assertFilterMatches(
new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
Arrays.asList(Interval.parse("1970-01-01T02:00:00.001Z/1970-01-01T02:00:00.005Z")),
exFn
),
ImmutableList.<String>of("1", "2", "3", "4")
);
}
@Test
public void testIntervalFilterOnStringDimension()
{
assertFilterMatches(
new IntervalDimFilter(
"dim0",
Arrays.asList(Interval.parse("1970-01-01T00:00:00.001Z/1970-01-01T00:00:00.005Z")),
null
),
ImmutableList.<String>of("1", "2", "3", "4")
);
assertFilterMatches(
new IntervalDimFilter(
"dim0",
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.000Z/1970-01-01T00:00:00.003Z"),
Interval.parse("1970-01-01T00:00:00.004Z/1970-01-01T00:00:00.006Z")
),
null
),
ImmutableList.<String>of("0", "1", "2", "4", "5")
);
assertFilterMatches(
new IntervalDimFilter(
"dim0",
Arrays.asList(
Interval.parse("1970-01-01T00:00:00.000Z/1970-01-01T00:00:00.001Z"),
Interval.parse("1970-01-01T00:00:00.003Z/1970-01-01T00:00:00.006Z"),
Interval.parse("1970-01-01T00:00:00.002Z/1970-01-01T00:00:00.005Z")
),
null
),
ImmutableList.<String>of("0", "2", "3", "4", "5")
);
assertFilterMatches(
new IntervalDimFilter(
"dim1",
Arrays.asList(Interval.parse("1970-01-01T00:00:00.002Z/1970-01-01T00:00:00.011Z")),
null
),
ImmutableList.<String>of("1", "2")
);
// increment timestamp by 2 hours
String timeBoosterJsFn = "function(x) { return(Number(x) + 7200000) }";
ExtractionFn exFn = new JavaScriptExtractionFn(timeBoosterJsFn, true, JavaScriptConfig.getDefault());
assertFilterMatches(
new IntervalDimFilter(
"dim0",
Arrays.asList(Interval.parse("1970-01-01T02:00:00.001Z/1970-01-01T02:00:00.005Z")),
exFn
),
ImmutableList.<String>of("1", "2", "3", "4")
);
}
private void assertFilterMatches(
final DimFilter filter,
final List<String> expectedRows