mirror of https://github.com/apache/druid.git
Comparing dimensions to each other in a filter (#3928)
Comparing dimensions to each other using a select filter
This commit is contained in:
parent
4b5ae31207
commit
2cbc4764f8
|
@ -18,6 +18,18 @@ 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.
|
The selector filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
||||||
|
|
||||||
|
### Column Comparison filter
|
||||||
|
|
||||||
|
The column comparison filter is similar to the selector filter, but instead compares dimensions to each other. For example:
|
||||||
|
|
||||||
|
``` json
|
||||||
|
"filter": { "type": "columnComparison", "dimensions": [<dimension_a>, <dimension_b>] }
|
||||||
|
```
|
||||||
|
|
||||||
|
This is the equivalent of `WHERE <dimension_a> = <dimension_b>`.
|
||||||
|
|
||||||
|
`dimensions` is list of [DimensionSpecs](./dimensionspecs.html), making it possible to apply an extraction function if needed.
|
||||||
|
|
||||||
### 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).
|
||||||
|
|
|
@ -29,6 +29,7 @@ dimensions. Filters follow these rules on multi-value dimensions:
|
||||||
|
|
||||||
- Value filters (like "selector", "bound", and "in") match a row if any of the values of a multi-value dimension match
|
- Value filters (like "selector", "bound", and "in") match a row if any of the values of a multi-value dimension match
|
||||||
the filter.
|
the filter.
|
||||||
|
- The Column Comparison filter will match a row if the dimensions have any overlap.
|
||||||
- Value filters that match `null` or `""` (empty string) will match empty cells in a multi-value dimension.
|
- Value filters that match `null` or `""` (empty string) will match empty cells in a multi-value dimension.
|
||||||
- Logical expression filters behave the same way they do on single-value dimensions: "and" matches a row if all
|
- Logical expression filters behave the same way they do on single-value dimensions: "and" matches a row if all
|
||||||
underlying filters match that row; "or" matches a row if any underlying filters match that row; "not" matches a row
|
underlying filters match that row; "or" matches a row if any underlying filters match that row; "not" matches a row
|
||||||
|
|
|
@ -0,0 +1,111 @@
|
||||||
|
/*
|
||||||
|
* 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.Joiner;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.collect.RangeSet;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
|
import io.druid.segment.filter.ColumnComparisonFilter;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class ColumnComparisonDimFilter implements DimFilter
|
||||||
|
{
|
||||||
|
private static final Joiner COMMA_JOINER = Joiner.on(", ");
|
||||||
|
|
||||||
|
private final List<DimensionSpec> dimensions;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ColumnComparisonDimFilter(
|
||||||
|
@JsonProperty("dimensions") List<DimensionSpec> dimensions
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.dimensions = Preconditions.checkNotNull(dimensions, "dimensions");
|
||||||
|
Preconditions.checkArgument(dimensions.size() >= 2, "dimensions must have a least 2 dimensions");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(DimFilterUtils.COLUMN_COMPARISON_CACHE_ID)
|
||||||
|
// Since a = b is the same as b = a we can ignore the order here.
|
||||||
|
.appendCacheablesIgnoringOrder(dimensions)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DimFilter optimize()
|
||||||
|
{
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Filter toFilter()
|
||||||
|
{
|
||||||
|
return new ColumnComparisonFilter(dimensions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public List<DimensionSpec> getDimensions()
|
||||||
|
{
|
||||||
|
return dimensions;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "ColumnComparisonDimFilter{" +
|
||||||
|
"dimensions=[" + COMMA_JOINER.join(dimensions) + "]" +
|
||||||
|
"}";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnComparisonDimFilter that = (ColumnComparisonDimFilter) o;
|
||||||
|
|
||||||
|
return dimensions.equals(that.dimensions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RangeSet<String> getDimensionRangeSet(String dimension)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return 31 * dimensions.hashCode();
|
||||||
|
}
|
||||||
|
}
|
|
@ -32,6 +32,7 @@ import io.druid.java.util.common.Cacheable;
|
||||||
@JsonSubTypes.Type(name="or", value=OrDimFilter.class),
|
@JsonSubTypes.Type(name="or", value=OrDimFilter.class),
|
||||||
@JsonSubTypes.Type(name="not", value=NotDimFilter.class),
|
@JsonSubTypes.Type(name="not", value=NotDimFilter.class),
|
||||||
@JsonSubTypes.Type(name="selector", value=SelectorDimFilter.class),
|
@JsonSubTypes.Type(name="selector", value=SelectorDimFilter.class),
|
||||||
|
@JsonSubTypes.Type(name="columnComparison", value=ColumnComparisonDimFilter.class),
|
||||||
@JsonSubTypes.Type(name="extraction", value=ExtractionDimFilter.class),
|
@JsonSubTypes.Type(name="extraction", value=ExtractionDimFilter.class),
|
||||||
@JsonSubTypes.Type(name="regex", value=RegexDimFilter.class),
|
@JsonSubTypes.Type(name="regex", value=RegexDimFilter.class),
|
||||||
@JsonSubTypes.Type(name="search", value=SearchQueryDimFilter.class),
|
@JsonSubTypes.Type(name="search", value=SearchQueryDimFilter.class),
|
||||||
|
|
|
@ -50,6 +50,7 @@ public class DimFilterUtils
|
||||||
static final byte BOUND_CACHE_ID = 0xA;
|
static final byte BOUND_CACHE_ID = 0xA;
|
||||||
static final byte INTERVAL_CACHE_ID = 0xB;
|
static final byte INTERVAL_CACHE_ID = 0xB;
|
||||||
static final byte LIKE_CACHE_ID = 0xC;
|
static final byte LIKE_CACHE_ID = 0xC;
|
||||||
|
static final byte COLUMN_COMPARISON_CACHE_ID = 0xD;
|
||||||
public static final byte STRING_SEPARATOR = (byte) 0xFF;
|
public static final byte STRING_SEPARATOR = (byte) 0xFF;
|
||||||
|
|
||||||
static byte[] computeCacheKey(byte cacheIdKey, List<DimFilter> filters)
|
static byte[] computeCacheKey(byte cacheIdKey, List<DimFilter> filters)
|
||||||
|
|
|
@ -59,4 +59,17 @@ public class FloatValueMatcherColumnSelectorStrategy implements ValueMatcherColu
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueGetter makeValueGetter(final FloatColumnSelector selector)
|
||||||
|
{
|
||||||
|
return new ValueGetter()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String[] get()
|
||||||
|
{
|
||||||
|
return new String[]{ Float.toString(selector.get()) };
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -58,4 +58,17 @@ public class LongValueMatcherColumnSelectorStrategy implements ValueMatcherColum
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueGetter makeValueGetter(final LongColumnSelector selector)
|
||||||
|
{
|
||||||
|
return new ValueGetter()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String[] get()
|
||||||
|
{
|
||||||
|
return new String[]{ Long.toString(selector.get()) };
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,10 +22,21 @@ package io.druid.query.filter;
|
||||||
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 io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.filter.BooleanValueMatcher;
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
|
|
||||||
public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy<DimensionSelector>
|
public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy<DimensionSelector>
|
||||||
{
|
{
|
||||||
|
private static final String[] NULL_VALUE = new String[]{ null };
|
||||||
|
private static final ValueGetter NULL_VALUE_GETTER = new ValueGetter()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String[] get()
|
||||||
|
{
|
||||||
|
return NULL_VALUE;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeValueMatcher(final DimensionSelector selector, String value)
|
public ValueMatcher makeValueMatcher(final DimensionSelector selector, String value)
|
||||||
{
|
{
|
||||||
|
@ -51,4 +62,30 @@ public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherCol
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueGetter makeValueGetter(final DimensionSelector selector)
|
||||||
|
{
|
||||||
|
if (selector.getValueCardinality() == 0) {
|
||||||
|
return NULL_VALUE_GETTER;
|
||||||
|
} else {
|
||||||
|
return new ValueGetter()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String[] get()
|
||||||
|
{
|
||||||
|
final IndexedInts row = selector.getRow();
|
||||||
|
final int size = row.size();
|
||||||
|
if (size == 0) {
|
||||||
|
return NULL_VALUE;
|
||||||
|
} else {
|
||||||
|
String[] values = new String[size];
|
||||||
|
for (int i = 0; i < size; ++i) {
|
||||||
|
values[i] = Strings.emptyToNull(selector.lookupName(row.get(i)));
|
||||||
|
}
|
||||||
|
return values;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,31 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public interface ValueGetter
|
||||||
|
{
|
||||||
|
// It is not ideal that Long and Float values will get
|
||||||
|
// converted to strings. We should also add functions
|
||||||
|
// for these and modify ColumnComparisonFilter to handle
|
||||||
|
// comparing Long and Float columns to eachother.
|
||||||
|
public String[] get();
|
||||||
|
}
|
|
@ -41,4 +41,12 @@ public interface ValueMatcherColumnSelectorStrategy<ValueSelectorType extends Co
|
||||||
* @return A ValueMatcher that applies the predicate for this DimensionQueryHelper's value type from the predicateFactory
|
* @return A ValueMatcher that applies the predicate for this DimensionQueryHelper's value type from the predicateFactory
|
||||||
*/
|
*/
|
||||||
ValueMatcher makeValueMatcher(ValueSelectorType selector, DruidPredicateFactory predicateFactory);
|
ValueMatcher makeValueMatcher(ValueSelectorType selector, DruidPredicateFactory predicateFactory);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a ValueGetter.
|
||||||
|
*
|
||||||
|
* @param selector Column selector
|
||||||
|
* @return A ValueGetter that returns the value(s) of the selected column
|
||||||
|
*/
|
||||||
|
ValueGetter makeValueGetter(ValueSelectorType selector);
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,144 @@
|
||||||
|
/*
|
||||||
|
* 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.Preconditions;
|
||||||
|
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||||
|
import io.druid.query.ColumnSelectorPlus;
|
||||||
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
|
import io.druid.query.filter.BitmapIndexSelector;
|
||||||
|
import io.druid.query.filter.Filter;
|
||||||
|
import io.druid.query.filter.ValueGetter;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
import io.druid.query.filter.ValueMatcherColumnSelectorStrategy;
|
||||||
|
import io.druid.query.filter.ValueMatcherColumnSelectorStrategyFactory;
|
||||||
|
import io.druid.segment.ColumnSelector;
|
||||||
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
|
import io.druid.segment.DimensionHandlerUtils;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class ColumnComparisonFilter implements Filter
|
||||||
|
{
|
||||||
|
private final List<DimensionSpec> dimensions;
|
||||||
|
|
||||||
|
public ColumnComparisonFilter(
|
||||||
|
final List<DimensionSpec> dimensions
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.dimensions = Preconditions.checkNotNull(dimensions, "dimensions");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
|
||||||
|
{
|
||||||
|
final ValueGetter[] valueGetters = new ValueGetter[dimensions.size()];
|
||||||
|
|
||||||
|
for (int i = 0; i < dimensions.size(); i++) {
|
||||||
|
final ColumnSelectorPlus<ValueMatcherColumnSelectorStrategy> selector =
|
||||||
|
DimensionHandlerUtils.createColumnSelectorPlus(
|
||||||
|
ValueMatcherColumnSelectorStrategyFactory.instance(),
|
||||||
|
dimensions.get(i),
|
||||||
|
factory
|
||||||
|
);
|
||||||
|
|
||||||
|
valueGetters[i] = selector.getColumnSelectorStrategy().makeValueGetter(selector.getSelector());
|
||||||
|
}
|
||||||
|
|
||||||
|
return makeValueMatcher(valueGetters);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ValueMatcher makeValueMatcher(final ValueGetter[] valueGetters) {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
// Keep all values to compare against each other.
|
||||||
|
String[][] values = new String[valueGetters.length][];
|
||||||
|
|
||||||
|
for (int i = 0; i < valueGetters.length; i++) {
|
||||||
|
values[i] = valueGetters[i].get();
|
||||||
|
// Compare the new values to the values we already got.
|
||||||
|
for (int j = 0; j < i; j++) {
|
||||||
|
if (!overlap(values[i], values[j])) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
// overlap returns true when: a and b have one or more elements in common,
|
||||||
|
// a and b are both null, or a and b are both empty.
|
||||||
|
public static boolean overlap(String[] a, String[] b) {
|
||||||
|
if (a == null || b == null) {
|
||||||
|
// They only have overlap if both are null.
|
||||||
|
return a == b;
|
||||||
|
}
|
||||||
|
if (a.length == 0 && b.length == 0) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int i = 0; i < a.length; i++) {
|
||||||
|
for (int j = 0; j < b.length; j++) {
|
||||||
|
if (a[i] == null || b[j] == null) {
|
||||||
|
if (a[i] == b[j]) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
} else if (a[i].equals(b[j])) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean supportsSelectivityEstimation(
|
||||||
|
ColumnSelector columnSelector, BitmapIndexSelector indexSelector
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double estimateSelectivity(BitmapIndexSelector indexSelector)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,91 @@
|
||||||
|
/*
|
||||||
|
* 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.google.common.collect.ImmutableList;
|
||||||
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
public class ColumnComparisonDimFilterTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void testGetCacheKey()
|
||||||
|
{
|
||||||
|
ColumnComparisonDimFilter columnComparisonDimFilter = new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec> of(
|
||||||
|
DefaultDimensionSpec.of("abc"),
|
||||||
|
DefaultDimensionSpec.of("d")
|
||||||
|
));
|
||||||
|
ColumnComparisonDimFilter columnComparisonDimFilter2 = new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec> of(
|
||||||
|
DefaultDimensionSpec.of("d"),
|
||||||
|
DefaultDimensionSpec.of("abc")
|
||||||
|
));
|
||||||
|
ColumnComparisonDimFilter columnComparisonDimFilter3 = new ColumnComparisonDimFilter(
|
||||||
|
ImmutableList.<DimensionSpec> of(
|
||||||
|
DefaultDimensionSpec.of("d"),
|
||||||
|
DefaultDimensionSpec.of("e")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertTrue(Arrays.equals(
|
||||||
|
columnComparisonDimFilter.getCacheKey(),
|
||||||
|
columnComparisonDimFilter2.getCacheKey()
|
||||||
|
));
|
||||||
|
Assert.assertFalse(Arrays.equals(
|
||||||
|
columnComparisonDimFilter2.getCacheKey(),
|
||||||
|
columnComparisonDimFilter3.getCacheKey()
|
||||||
|
));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testHashCode()
|
||||||
|
{
|
||||||
|
ColumnComparisonDimFilter columnComparisonDimFilter = new ColumnComparisonDimFilter(
|
||||||
|
ImmutableList.<DimensionSpec> of(
|
||||||
|
DefaultDimensionSpec.of("abc"),
|
||||||
|
DefaultDimensionSpec.of("d")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
ColumnComparisonDimFilter columnComparisonDimFilter2 = new ColumnComparisonDimFilter(
|
||||||
|
ImmutableList.<DimensionSpec> of(
|
||||||
|
DefaultDimensionSpec.of("d"),
|
||||||
|
DefaultDimensionSpec.of("abc")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
ColumnComparisonDimFilter columnComparisonDimFilter3 = new ColumnComparisonDimFilter(
|
||||||
|
ImmutableList.<DimensionSpec> of(
|
||||||
|
DefaultDimensionSpec.of("d"),
|
||||||
|
DefaultDimensionSpec.of("e")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertNotEquals(
|
||||||
|
columnComparisonDimFilter.hashCode(),
|
||||||
|
columnComparisonDimFilter2.hashCode()
|
||||||
|
);
|
||||||
|
Assert.assertNotEquals(
|
||||||
|
columnComparisonDimFilter2.hashCode(),
|
||||||
|
columnComparisonDimFilter3.hashCode()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,150 @@
|
||||||
|
/*
|
||||||
|
* 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 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.java.util.common.Pair;
|
||||||
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
|
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||||
|
import io.druid.query.extraction.MapLookupExtractor;
|
||||||
|
import io.druid.query.filter.ColumnComparisonDimFilter;
|
||||||
|
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.AfterClass;
|
||||||
|
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 ColumnComparisonFilterTest 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")),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "0", "dim1", "", "dim2", ImmutableList.of("1", "2"))),
|
||||||
|
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("3"))),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "4", "dim1", "1", "dim2", ImmutableList.of("4", "5"))),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "5", "dim1", "5", "dim2", ImmutableList.of("4", "5"))),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "6", "dim1", "1")),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "7", "dim1", "a")),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "8", "dim1", 8L)),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "9", "dim1", 1.234f, "dim2", 1.234f))
|
||||||
|
);
|
||||||
|
|
||||||
|
public ColumnComparisonFilterTest(
|
||||||
|
String testName,
|
||||||
|
IndexBuilder indexBuilder,
|
||||||
|
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||||
|
boolean cnf,
|
||||||
|
boolean optimize
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void tearDown() throws Exception
|
||||||
|
{
|
||||||
|
BaseFilterTest.tearDown(ColumnComparisonFilterTest.class.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testColumnsWithoutNulls()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec>of(
|
||||||
|
DefaultDimensionSpec.of("dim0"),
|
||||||
|
DefaultDimensionSpec.of("dim1")
|
||||||
|
)), ImmutableList.<String>of("2","5","8"));
|
||||||
|
assertFilterMatches(new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec>of(
|
||||||
|
DefaultDimensionSpec.of("dim0"),
|
||||||
|
DefaultDimensionSpec.of("dim2")
|
||||||
|
)), ImmutableList.<String>of("3","4","5"));
|
||||||
|
assertFilterMatches(new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec>of(
|
||||||
|
DefaultDimensionSpec.of("dim1"),
|
||||||
|
DefaultDimensionSpec.of("dim2")
|
||||||
|
)), ImmutableList.<String>of("5","9"));
|
||||||
|
assertFilterMatches(new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec>of(
|
||||||
|
DefaultDimensionSpec.of("dim0"),
|
||||||
|
DefaultDimensionSpec.of("dim1"),
|
||||||
|
DefaultDimensionSpec.of("dim2")
|
||||||
|
)), ImmutableList.<String>of("5"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMissingColumnNotSpecifiedInDimensionList()
|
||||||
|
{
|
||||||
|
assertFilterMatches(new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec>of(
|
||||||
|
DefaultDimensionSpec.of("dim6"),
|
||||||
|
DefaultDimensionSpec.of("dim7")
|
||||||
|
)), ImmutableList.<String>of("0","1","2","3","4","5","6","7","8","9"));
|
||||||
|
assertFilterMatches(new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec>of(
|
||||||
|
DefaultDimensionSpec.of("dim1"),
|
||||||
|
DefaultDimensionSpec.of("dim6")
|
||||||
|
)), ImmutableList.<String>of("0"));
|
||||||
|
assertFilterMatches(new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec>of(
|
||||||
|
DefaultDimensionSpec.of("dim2"),
|
||||||
|
DefaultDimensionSpec.of("dim6")
|
||||||
|
)), ImmutableList.<String>of("1","2","6","7","8"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSelectorWithLookupExtractionFn()
|
||||||
|
{
|
||||||
|
final Map<String, String> stringMap = ImmutableMap.of(
|
||||||
|
"a", "7"
|
||||||
|
);
|
||||||
|
LookupExtractor mapExtractor = new MapLookupExtractor(stringMap, false);
|
||||||
|
LookupExtractionFn lookupFn = new LookupExtractionFn(mapExtractor, true, null, false, true);
|
||||||
|
|
||||||
|
assertFilterMatches(new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec>of(
|
||||||
|
new ExtractionDimensionSpec("dim0", "dim0", lookupFn),
|
||||||
|
new ExtractionDimensionSpec("dim1", "dim1", lookupFn)
|
||||||
|
)), ImmutableList.<String>of("2","5","7","8"));
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue