Comparing dimensions to each other in a filter (#3928)

Comparing dimensions to each other using a select filter
This commit is contained in:
Erik Dubbelboer 2017-03-24 09:23:46 +08:00 committed by Jonathan Wei
parent 4b5ae31207
commit 2cbc4764f8
13 changed files with 613 additions and 0 deletions

View File

@ -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.
### 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
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).

View File

@ -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
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.
- 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

View File

@ -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();
}
}

View File

@ -32,6 +32,7 @@ import io.druid.java.util.common.Cacheable;
@JsonSubTypes.Type(name="or", value=OrDimFilter.class),
@JsonSubTypes.Type(name="not", value=NotDimFilter.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="regex", value=RegexDimFilter.class),
@JsonSubTypes.Type(name="search", value=SearchQueryDimFilter.class),

View File

@ -50,6 +50,7 @@ public class DimFilterUtils
static final byte BOUND_CACHE_ID = 0xA;
static final byte INTERVAL_CACHE_ID = 0xB;
static final byte LIKE_CACHE_ID = 0xC;
static final byte COLUMN_COMPARISON_CACHE_ID = 0xD;
public static final byte STRING_SEPARATOR = (byte) 0xFF;
static byte[] computeCacheKey(byte cacheIdKey, List<DimFilter> filters)

View File

@ -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()) };
}
};
}
}

View File

@ -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()) };
}
};
}
}

View File

@ -22,10 +22,21 @@ package io.druid.query.filter;
import com.google.common.base.Predicate;
import com.google.common.base.Strings;
import io.druid.segment.DimensionSelector;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.filter.BooleanValueMatcher;
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
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;
}
}
};
}
}
}

View File

@ -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();
}

View File

@ -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
*/
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);
}

View File

@ -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();
}
}

View File

@ -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()
);
}
}

View File

@ -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"));
}
}