mirror of https://github.com/apache/druid.git
Add "like" filter. (#3642)
* Add "like" filter. * Addressed some PR comments. * Slight simplifications to LikeFilter. * Additional simplifications. * Fix comment in LikeFilter. * Clarify comment in LikeFilter. * Simplify LikeMatcher a bit. * No use going through the optimized path if prefix is empty. * Add more tests.
This commit is contained in:
parent
b961b6a69f
commit
2c504b6258
|
@ -0,0 +1,251 @@
|
||||||
|
/*
|
||||||
|
* 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.benchmark;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.collect.FluentIterable;
|
||||||
|
import com.metamx.collections.bitmap.BitmapFactory;
|
||||||
|
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||||
|
import com.metamx.collections.bitmap.MutableBitmap;
|
||||||
|
import com.metamx.collections.bitmap.RoaringBitmapFactory;
|
||||||
|
import com.metamx.collections.spatial.ImmutableRTree;
|
||||||
|
import io.druid.query.filter.BitmapIndexSelector;
|
||||||
|
import io.druid.query.filter.BoundDimFilter;
|
||||||
|
import io.druid.query.filter.Filter;
|
||||||
|
import io.druid.query.filter.LikeDimFilter;
|
||||||
|
import io.druid.query.filter.RegexDimFilter;
|
||||||
|
import io.druid.query.filter.SelectorDimFilter;
|
||||||
|
import io.druid.query.ordering.StringComparators;
|
||||||
|
import io.druid.segment.column.BitmapIndex;
|
||||||
|
import io.druid.segment.data.BitmapSerdeFactory;
|
||||||
|
import io.druid.segment.data.GenericIndexed;
|
||||||
|
import io.druid.segment.data.Indexed;
|
||||||
|
import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||||
|
import io.druid.segment.serde.BitmapIndexColumnPartSupplier;
|
||||||
|
import org.openjdk.jmh.annotations.Benchmark;
|
||||||
|
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||||
|
import org.openjdk.jmh.annotations.Fork;
|
||||||
|
import org.openjdk.jmh.annotations.Measurement;
|
||||||
|
import org.openjdk.jmh.annotations.Mode;
|
||||||
|
import org.openjdk.jmh.annotations.OutputTimeUnit;
|
||||||
|
import org.openjdk.jmh.annotations.Param;
|
||||||
|
import org.openjdk.jmh.annotations.Scope;
|
||||||
|
import org.openjdk.jmh.annotations.Setup;
|
||||||
|
import org.openjdk.jmh.annotations.State;
|
||||||
|
import org.openjdk.jmh.annotations.Warmup;
|
||||||
|
import org.openjdk.jmh.infra.Blackhole;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
@State(Scope.Benchmark)
|
||||||
|
@Fork(value = 1)
|
||||||
|
@Warmup(iterations = 10)
|
||||||
|
@Measurement(iterations = 10)
|
||||||
|
public class LikeFilterBenchmark
|
||||||
|
{
|
||||||
|
private static final int START_INT = 1_000_000;
|
||||||
|
private static final int END_INT = 9_999_999;
|
||||||
|
|
||||||
|
private static final Filter SELECTOR_EQUALS = new SelectorDimFilter(
|
||||||
|
"foo",
|
||||||
|
"1000000",
|
||||||
|
null
|
||||||
|
).toFilter();
|
||||||
|
|
||||||
|
private static final Filter LIKE_EQUALS = new LikeDimFilter(
|
||||||
|
"foo",
|
||||||
|
"1000000",
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
).toFilter();
|
||||||
|
|
||||||
|
private static final Filter BOUND_PREFIX = new BoundDimFilter(
|
||||||
|
"foo",
|
||||||
|
"50",
|
||||||
|
"50\uffff",
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
StringComparators.LEXICOGRAPHIC
|
||||||
|
).toFilter();
|
||||||
|
|
||||||
|
private static final Filter REGEX_PREFIX = new RegexDimFilter(
|
||||||
|
"foo",
|
||||||
|
"^50.*",
|
||||||
|
null
|
||||||
|
).toFilter();
|
||||||
|
|
||||||
|
private static final Filter LIKE_PREFIX = new LikeDimFilter(
|
||||||
|
"foo",
|
||||||
|
"50%",
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
).toFilter();
|
||||||
|
|
||||||
|
// cardinality, the dictionary will contain evenly spaced integers
|
||||||
|
@Param({"1000", "100000", "1000000"})
|
||||||
|
int cardinality;
|
||||||
|
|
||||||
|
int step;
|
||||||
|
|
||||||
|
// selector will contain a cardinality number of bitmaps; each one contains a single int: 0
|
||||||
|
BitmapIndexSelector selector;
|
||||||
|
|
||||||
|
@Setup
|
||||||
|
public void setup() throws IOException
|
||||||
|
{
|
||||||
|
step = (END_INT - START_INT) / cardinality;
|
||||||
|
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||||
|
final BitmapSerdeFactory serdeFactory = new RoaringBitmapSerdeFactory(null);
|
||||||
|
final List<Integer> ints = generateInts();
|
||||||
|
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||||
|
FluentIterable.from(ints)
|
||||||
|
.transform(
|
||||||
|
new Function<Integer, String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String apply(Integer i)
|
||||||
|
{
|
||||||
|
return i.toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
),
|
||||||
|
GenericIndexed.STRING_STRATEGY
|
||||||
|
);
|
||||||
|
final BitmapIndex bitmapIndex = new BitmapIndexColumnPartSupplier(
|
||||||
|
bitmapFactory,
|
||||||
|
GenericIndexed.fromIterable(
|
||||||
|
FluentIterable.from(ints)
|
||||||
|
.transform(
|
||||||
|
new Function<Integer, ImmutableBitmap>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public ImmutableBitmap apply(Integer i)
|
||||||
|
{
|
||||||
|
final MutableBitmap mutableBitmap = bitmapFactory.makeEmptyMutableBitmap();
|
||||||
|
mutableBitmap.add((i - START_INT) / step);
|
||||||
|
return bitmapFactory.makeImmutableBitmap(mutableBitmap);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
),
|
||||||
|
serdeFactory.getObjectStrategy()
|
||||||
|
),
|
||||||
|
dictionary
|
||||||
|
).get();
|
||||||
|
selector = new BitmapIndexSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Indexed<String> getDimensionValues(String dimension)
|
||||||
|
{
|
||||||
|
return dictionary;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getNumRows()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BitmapFactory getBitmapFactory()
|
||||||
|
{
|
||||||
|
return bitmapFactory;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ImmutableBitmap getBitmapIndex(String dimension, String value)
|
||||||
|
{
|
||||||
|
return bitmapIndex.getBitmap(bitmapIndex.getIndex(value));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BitmapIndex getBitmapIndex(String dimension)
|
||||||
|
{
|
||||||
|
return bitmapIndex;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ImmutableRTree getSpatialIndex(String dimension)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Benchmark
|
||||||
|
@BenchmarkMode(Mode.AverageTime)
|
||||||
|
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||||
|
public void matchLikeEquals(Blackhole blackhole)
|
||||||
|
{
|
||||||
|
final ImmutableBitmap bitmapIndex = LIKE_EQUALS.getBitmapIndex(selector);
|
||||||
|
blackhole.consume(bitmapIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Benchmark
|
||||||
|
@BenchmarkMode(Mode.AverageTime)
|
||||||
|
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||||
|
public void matchSelectorEquals(Blackhole blackhole)
|
||||||
|
{
|
||||||
|
final ImmutableBitmap bitmapIndex = SELECTOR_EQUALS.getBitmapIndex(selector);
|
||||||
|
blackhole.consume(bitmapIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Benchmark
|
||||||
|
@BenchmarkMode(Mode.AverageTime)
|
||||||
|
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||||
|
public void matchLikePrefix(Blackhole blackhole)
|
||||||
|
{
|
||||||
|
final ImmutableBitmap bitmapIndex = LIKE_PREFIX.getBitmapIndex(selector);
|
||||||
|
blackhole.consume(bitmapIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Benchmark
|
||||||
|
@BenchmarkMode(Mode.AverageTime)
|
||||||
|
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||||
|
public void matchBoundPrefix(Blackhole blackhole)
|
||||||
|
{
|
||||||
|
final ImmutableBitmap bitmapIndex = BOUND_PREFIX.getBitmapIndex(selector);
|
||||||
|
blackhole.consume(bitmapIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Benchmark
|
||||||
|
@BenchmarkMode(Mode.AverageTime)
|
||||||
|
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||||
|
public void matchRegexPrefix(Blackhole blackhole)
|
||||||
|
{
|
||||||
|
final ImmutableBitmap bitmapIndex = REGEX_PREFIX.getBitmapIndex(selector);
|
||||||
|
blackhole.consume(bitmapIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<Integer> generateInts()
|
||||||
|
{
|
||||||
|
final List<Integer> ints = new ArrayList<>(cardinality);
|
||||||
|
|
||||||
|
for (int i = 0; i < cardinality; i++) {
|
||||||
|
ints.add(START_INT + step * i);
|
||||||
|
}
|
||||||
|
|
||||||
|
return ints;
|
||||||
|
}
|
||||||
|
}
|
|
@ -202,10 +202,36 @@ 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.
|
The IN filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
||||||
|
|
||||||
|
### Like filter
|
||||||
|
|
||||||
|
Like filters can be used for basic wildcard searches. They are equivalent to the SQL LIKE operator. Special characters
|
||||||
|
supported are "%" (matches any number of characters) and "\_" (matches any one character).
|
||||||
|
|
||||||
|
|property|type|description|required?|
|
||||||
|
|--------|-----------|---------|---------|
|
||||||
|
|type|String|This should always be "like".|yes|
|
||||||
|
|dimension|String|The dimension to filter on|yes|
|
||||||
|
|pattern|String|LIKE pattern, such as "foo%" or "___bar".|yes|
|
||||||
|
|escape|String|An escape character that can be used to escape special characters.|no|
|
||||||
|
|extractionFn|[Extraction function](#filtering-with-extraction-functions)| Extraction function to apply to the dimension|no|
|
||||||
|
|
||||||
|
Like filters support the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
||||||
|
|
||||||
|
This Like filter expresses the condition `last_name LIKE "D%"` (i.e. last_name starts with "D").
|
||||||
|
|
||||||
|
```json
|
||||||
|
{
|
||||||
|
"type": "like",
|
||||||
|
"dimension": "last_name",
|
||||||
|
"pattern": "D%"
|
||||||
|
}
|
||||||
|
```
|
||||||
|
|
||||||
### Bound filter
|
### Bound filter
|
||||||
|
|
||||||
The Bound filter can be used to filter by comparing dimension values to an upper value and/or a lower value.
|
Bound filters can be used to filter on ranges of dimension values. It can be used for comparison filtering like
|
||||||
|
greater than, less than, greater than or equal to, less than or equal to, and "between" (if both "lower" and
|
||||||
|
"upper" are set).
|
||||||
|
|
||||||
|property|type|description|required?|
|
|property|type|description|required?|
|
||||||
|--------|-----------|---------|---------|
|
|--------|-----------|---------|---------|
|
||||||
|
@ -218,7 +244,7 @@ The Bound filter can be used to filter by comparing dimension values to an upper
|
||||||
|ordering|String|Specifies the sorting order to use when comparing values against the bound. Can be one of the following values: "lexicographic", "alphanumeric", "numeric", "strlen". See [Sorting Orders](./sorting-orders.html) for more details.|no, default: "lexicographic"|
|
|ordering|String|Specifies the sorting order to use when comparing values against the bound. Can be one of the following values: "lexicographic", "alphanumeric", "numeric", "strlen". See [Sorting Orders](./sorting-orders.html) for more details.|no, default: "lexicographic"|
|
||||||
|extractionFn|[Extraction function](#filtering-with-extraction-functions)| Extraction function to apply to the dimension|no|
|
|extractionFn|[Extraction function](#filtering-with-extraction-functions)| Extraction function to apply to the dimension|no|
|
||||||
|
|
||||||
The bound filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
Bound filters support the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
|
||||||
|
|
||||||
The following bound filter expresses the condition `21 <= age <= 31`:
|
The following bound filter expresses the condition `21 <= age <= 31`:
|
||||||
```json
|
```json
|
||||||
|
|
|
@ -38,7 +38,8 @@ import com.google.common.collect.RangeSet;
|
||||||
@JsonSubTypes.Type(name="spatial", value=SpatialDimFilter.class),
|
@JsonSubTypes.Type(name="spatial", value=SpatialDimFilter.class),
|
||||||
@JsonSubTypes.Type(name="in", value=InDimFilter.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)
|
@JsonSubTypes.Type(name="interval", value=IntervalDimFilter.class),
|
||||||
|
@JsonSubTypes.Type(name="like", value=LikeDimFilter.class)
|
||||||
})
|
})
|
||||||
public interface DimFilter
|
public interface DimFilter
|
||||||
{
|
{
|
||||||
|
|
|
@ -49,6 +49,7 @@ public class DimFilterUtils
|
||||||
static final byte IN_CACHE_ID = 0x9;
|
static final byte IN_CACHE_ID = 0x9;
|
||||||
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;
|
||||||
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)
|
||||||
|
|
|
@ -0,0 +1,357 @@
|
||||||
|
/*
|
||||||
|
* 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.base.Predicate;
|
||||||
|
import com.google.common.base.Strings;
|
||||||
|
import com.google.common.collect.RangeSet;
|
||||||
|
import com.google.common.io.BaseEncoding;
|
||||||
|
import com.google.common.primitives.Chars;
|
||||||
|
import io.druid.java.util.common.StringUtils;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.segment.filter.LikeFilter;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
public class LikeDimFilter implements DimFilter
|
||||||
|
{
|
||||||
|
// Regex matching characters that are definitely okay to include unescaped in a regex.
|
||||||
|
// Leads to excessively paranoid escaping, although shouldn't affect runtime beyond compiling the regex.
|
||||||
|
private static final Pattern DEFINITELY_FINE = Pattern.compile("[\\w\\d\\s-]");
|
||||||
|
private static final String WILDCARD = ".*";
|
||||||
|
|
||||||
|
private final String dimension;
|
||||||
|
private final String pattern;
|
||||||
|
private final Character escapeChar;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
private final LikeMatcher likeMatcher;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public LikeDimFilter(
|
||||||
|
@JsonProperty("dimension") final String dimension,
|
||||||
|
@JsonProperty("pattern") final String pattern,
|
||||||
|
@JsonProperty("escape") final String escape,
|
||||||
|
@JsonProperty("extractionFn") final ExtractionFn extractionFn
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.dimension = Preconditions.checkNotNull(dimension, "dimension");
|
||||||
|
this.pattern = Preconditions.checkNotNull(pattern, "pattern");
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
|
|
||||||
|
if (escape != null && escape.length() != 1) {
|
||||||
|
throw new IllegalArgumentException("Escape must be null or a single character");
|
||||||
|
} else {
|
||||||
|
this.escapeChar = (escape == null || escape.isEmpty()) ? null : escape.charAt(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
this.likeMatcher = LikeMatcher.from(pattern, this.escapeChar);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class LikeMatcher
|
||||||
|
{
|
||||||
|
public enum SuffixMatch
|
||||||
|
{
|
||||||
|
MATCH_ANY,
|
||||||
|
MATCH_EMPTY,
|
||||||
|
MATCH_PATTERN
|
||||||
|
}
|
||||||
|
|
||||||
|
// Strings match if:
|
||||||
|
// (a) suffixMatch is MATCH_ANY and they start with "prefix"
|
||||||
|
// (b) suffixMatch is MATCH_EMPTY and they start with "prefix" and contain nothing after prefix
|
||||||
|
// (c) suffixMatch is MATCH_PATTERN and the string matches "pattern"
|
||||||
|
private final SuffixMatch suffixMatch;
|
||||||
|
|
||||||
|
// Prefix that matching strings are known to start with. May be empty.
|
||||||
|
private final String prefix;
|
||||||
|
|
||||||
|
// Regex pattern that describes matching strings.
|
||||||
|
private final Pattern pattern;
|
||||||
|
|
||||||
|
private LikeMatcher(
|
||||||
|
final SuffixMatch suffixMatch,
|
||||||
|
final String prefix,
|
||||||
|
final Pattern pattern
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.suffixMatch = Preconditions.checkNotNull(suffixMatch, "suffixMatch");
|
||||||
|
this.prefix = Strings.nullToEmpty(prefix);
|
||||||
|
this.pattern = Preconditions.checkNotNull(pattern, "pattern");
|
||||||
|
}
|
||||||
|
|
||||||
|
public static LikeMatcher from(
|
||||||
|
final String likePattern,
|
||||||
|
@Nullable final Character escapeChar
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final StringBuilder prefix = new StringBuilder();
|
||||||
|
final StringBuilder regex = new StringBuilder();
|
||||||
|
boolean escaping = false;
|
||||||
|
boolean inPrefix = true;
|
||||||
|
SuffixMatch suffixMatch = SuffixMatch.MATCH_EMPTY;
|
||||||
|
for (int i = 0; i < likePattern.length(); i++) {
|
||||||
|
final char c = likePattern.charAt(i);
|
||||||
|
if (escapeChar != null && c == escapeChar) {
|
||||||
|
escaping = true;
|
||||||
|
} else if (c == '%' && !escaping) {
|
||||||
|
inPrefix = false;
|
||||||
|
if (suffixMatch == SuffixMatch.MATCH_EMPTY) {
|
||||||
|
suffixMatch = SuffixMatch.MATCH_ANY;
|
||||||
|
}
|
||||||
|
regex.append(WILDCARD);
|
||||||
|
} else if (c == '_' && !escaping) {
|
||||||
|
inPrefix = false;
|
||||||
|
suffixMatch = SuffixMatch.MATCH_PATTERN;
|
||||||
|
regex.append(".");
|
||||||
|
} else {
|
||||||
|
if (inPrefix) {
|
||||||
|
prefix.append(c);
|
||||||
|
} else {
|
||||||
|
suffixMatch = SuffixMatch.MATCH_PATTERN;
|
||||||
|
}
|
||||||
|
addPatternCharacter(regex, c);
|
||||||
|
escaping = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return new LikeMatcher(suffixMatch, prefix.toString(), Pattern.compile(regex.toString()));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void addPatternCharacter(final StringBuilder patternBuilder, final char c)
|
||||||
|
{
|
||||||
|
if (DEFINITELY_FINE.matcher(String.valueOf(c)).matches()) {
|
||||||
|
patternBuilder.append(c);
|
||||||
|
} else {
|
||||||
|
patternBuilder.append("\\u").append(BaseEncoding.base16().encode(Chars.toByteArray(c)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean matches(@Nullable final String s)
|
||||||
|
{
|
||||||
|
return pattern.matcher(Strings.nullToEmpty(s)).matches();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks if the suffix of "s" matches the suffix of this matcher. The first prefix.length characters
|
||||||
|
* of s are ignored. This method is useful if you've already independently verified the prefix.
|
||||||
|
*/
|
||||||
|
public boolean matchesSuffixOnly(@Nullable final String s)
|
||||||
|
{
|
||||||
|
if (suffixMatch == SuffixMatch.MATCH_ANY) {
|
||||||
|
return true;
|
||||||
|
} else if (suffixMatch == SuffixMatch.MATCH_EMPTY) {
|
||||||
|
return (s == null ? 0 : s.length()) == prefix.length();
|
||||||
|
} else {
|
||||||
|
// suffixMatch is MATCH_PATTERN
|
||||||
|
return matches(s);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public DruidPredicateFactory predicateFactory(final ExtractionFn extractionFn)
|
||||||
|
{
|
||||||
|
return new DruidPredicateFactory()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Predicate<String> makeStringPredicate()
|
||||||
|
{
|
||||||
|
if (extractionFn != null) {
|
||||||
|
return new Predicate<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean apply(String input)
|
||||||
|
{
|
||||||
|
return matches(extractionFn.apply(input));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return new Predicate<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean apply(String input)
|
||||||
|
{
|
||||||
|
return matches(input);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DruidLongPredicate makeLongPredicate()
|
||||||
|
{
|
||||||
|
if (extractionFn != null) {
|
||||||
|
return new DruidLongPredicate()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean applyLong(long input)
|
||||||
|
{
|
||||||
|
return matches(extractionFn.apply(input));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return new DruidLongPredicate()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean applyLong(long input)
|
||||||
|
{
|
||||||
|
return matches(String.valueOf(input));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getPrefix()
|
||||||
|
{
|
||||||
|
return prefix;
|
||||||
|
}
|
||||||
|
|
||||||
|
public SuffixMatch getSuffixMatch()
|
||||||
|
{
|
||||||
|
return suffixMatch;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public String getDimension()
|
||||||
|
{
|
||||||
|
return dimension;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public String getPattern()
|
||||||
|
{
|
||||||
|
return pattern;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public String getEscape()
|
||||||
|
{
|
||||||
|
return escapeChar != null ? escapeChar.toString() : null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public ExtractionFn getExtractionFn()
|
||||||
|
{
|
||||||
|
return extractionFn;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
|
||||||
|
final byte[] patternBytes = StringUtils.toUtf8(pattern);
|
||||||
|
final byte[] escapeBytes = escapeChar == null ? new byte[0] : Chars.toByteArray(escapeChar);
|
||||||
|
final byte[] extractionFnBytes = extractionFn == null ? new byte[0] : extractionFn.getCacheKey();
|
||||||
|
final int sz = 4 + dimensionBytes.length + patternBytes.length + escapeBytes.length + extractionFnBytes.length;
|
||||||
|
return ByteBuffer.allocate(sz)
|
||||||
|
.put(DimFilterUtils.LIKE_CACHE_ID)
|
||||||
|
.put(dimensionBytes)
|
||||||
|
.put(DimFilterUtils.STRING_SEPARATOR)
|
||||||
|
.put(patternBytes)
|
||||||
|
.put(DimFilterUtils.STRING_SEPARATOR)
|
||||||
|
.put(escapeBytes)
|
||||||
|
.put(DimFilterUtils.STRING_SEPARATOR)
|
||||||
|
.put(extractionFnBytes)
|
||||||
|
.array();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DimFilter optimize()
|
||||||
|
{
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Filter toFilter()
|
||||||
|
{
|
||||||
|
return new LikeFilter(dimension, extractionFn, likeMatcher);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RangeSet<String> getDimensionRangeSet(String dimension)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
LikeDimFilter that = (LikeDimFilter) o;
|
||||||
|
|
||||||
|
if (dimension != null ? !dimension.equals(that.dimension) : that.dimension != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (pattern != null ? !pattern.equals(that.pattern) : that.pattern != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (escapeChar != null ? !escapeChar.equals(that.escapeChar) : that.escapeChar != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return extractionFn != null ? extractionFn.equals(that.extractionFn) : that.extractionFn == null;
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
int result = dimension != null ? dimension.hashCode() : 0;
|
||||||
|
result = 31 * result + (pattern != null ? pattern.hashCode() : 0);
|
||||||
|
result = 31 * result + (escapeChar != null ? escapeChar.hashCode() : 0);
|
||||||
|
result = 31 * result + (extractionFn != null ? extractionFn.hashCode() : 0);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
final StringBuilder builder = new StringBuilder();
|
||||||
|
|
||||||
|
if (extractionFn != null) {
|
||||||
|
builder.append(extractionFn).append("(");
|
||||||
|
}
|
||||||
|
|
||||||
|
builder.append(dimension);
|
||||||
|
|
||||||
|
if (extractionFn != null) {
|
||||||
|
builder.append(")");
|
||||||
|
}
|
||||||
|
|
||||||
|
builder.append(" LIKE '").append(pattern).append("'");
|
||||||
|
|
||||||
|
if (escapeChar != null) {
|
||||||
|
builder.append(" ESCAPE '").append(escapeChar).append("'");
|
||||||
|
}
|
||||||
|
|
||||||
|
return builder.toString();
|
||||||
|
}
|
||||||
|
}
|
|
@ -61,12 +61,7 @@ public class BoundFilter implements Filter
|
||||||
final BitmapIndex bitmapIndex = selector.getBitmapIndex(boundDimFilter.getDimension());
|
final BitmapIndex bitmapIndex = selector.getBitmapIndex(boundDimFilter.getDimension());
|
||||||
|
|
||||||
if (bitmapIndex == null || bitmapIndex.getCardinality() == 0) {
|
if (bitmapIndex == null || bitmapIndex.getCardinality() == 0) {
|
||||||
if (doesMatch(null)) {
|
return doesMatch(null) ? Filters.allTrue(selector) : Filters.allFalse(selector);
|
||||||
return selector.getBitmapFactory()
|
|
||||||
.complement(selector.getBitmapFactory().makeEmptyImmutableBitmap(), selector.getNumRows());
|
|
||||||
} else {
|
|
||||||
return selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// search for start, end indexes in the bitmaps; then include all bitmaps between those points
|
// search for start, end indexes in the bitmaps; then include all bitmaps between those points
|
||||||
|
|
|
@ -87,6 +87,17 @@ public class Filters
|
||||||
return dimFilter == null ? null : dimFilter.toFilter();
|
return dimFilter == null ? null : dimFilter.toFilter();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static ImmutableBitmap allFalse(final BitmapIndexSelector selector)
|
||||||
|
{
|
||||||
|
return selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ImmutableBitmap allTrue(final BitmapIndexSelector selector)
|
||||||
|
{
|
||||||
|
return selector.getBitmapFactory()
|
||||||
|
.complement(selector.getBitmapFactory().makeEmptyImmutableBitmap(), selector.getNumRows());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the union of bitmaps for all values matching a particular predicate.
|
* Return the union of bitmaps for all values matching a particular predicate.
|
||||||
*
|
*
|
||||||
|
|
|
@ -0,0 +1,143 @@
|
||||||
|
/*
|
||||||
|
* 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.Strings;
|
||||||
|
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.LikeDimFilter;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
import io.druid.query.filter.ValueMatcherFactory;
|
||||||
|
import io.druid.segment.column.BitmapIndex;
|
||||||
|
import io.druid.segment.data.Indexed;
|
||||||
|
|
||||||
|
import java.util.Iterator;
|
||||||
|
|
||||||
|
public class LikeFilter implements Filter
|
||||||
|
{
|
||||||
|
private final String dimension;
|
||||||
|
private final ExtractionFn extractionFn;
|
||||||
|
private final LikeDimFilter.LikeMatcher likeMatcher;
|
||||||
|
|
||||||
|
public LikeFilter(
|
||||||
|
final String dimension,
|
||||||
|
final ExtractionFn extractionFn,
|
||||||
|
final LikeDimFilter.LikeMatcher likeMatcher
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.dimension = dimension;
|
||||||
|
this.extractionFn = extractionFn;
|
||||||
|
this.likeMatcher = likeMatcher;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
|
||||||
|
{
|
||||||
|
if (extractionFn == null && likeMatcher.getSuffixMatch() == LikeDimFilter.LikeMatcher.SuffixMatch.MATCH_EMPTY) {
|
||||||
|
// dimension equals prefix
|
||||||
|
return selector.getBitmapIndex(dimension, likeMatcher.getPrefix());
|
||||||
|
} else if (extractionFn == null && !likeMatcher.getPrefix().isEmpty()) {
|
||||||
|
// dimension startsWith prefix and is accepted by likeMatcher.matchesSuffixOnly
|
||||||
|
final BitmapIndex bitmapIndex = selector.getBitmapIndex(dimension);
|
||||||
|
|
||||||
|
if (bitmapIndex == null) {
|
||||||
|
// Treat this as a column full of nulls
|
||||||
|
return likeMatcher.matches(null) ? Filters.allTrue(selector) : Filters.allFalse(selector);
|
||||||
|
}
|
||||||
|
|
||||||
|
// search for start, end indexes in the bitmaps; then include all matching bitmaps between those points
|
||||||
|
final Indexed<String> dimValues = selector.getDimensionValues(dimension);
|
||||||
|
|
||||||
|
final String lower = Strings.nullToEmpty(likeMatcher.getPrefix());
|
||||||
|
final String upper = Strings.nullToEmpty(likeMatcher.getPrefix()) + Character.MAX_VALUE;
|
||||||
|
final int startIndex; // inclusive
|
||||||
|
final int endIndex; // exclusive
|
||||||
|
|
||||||
|
final int lowerFound = bitmapIndex.getIndex(lower);
|
||||||
|
startIndex = lowerFound >= 0 ? lowerFound : -(lowerFound + 1);
|
||||||
|
|
||||||
|
final int upperFound = bitmapIndex.getIndex(upper);
|
||||||
|
endIndex = upperFound >= 0 ? upperFound + 1 : -(upperFound + 1);
|
||||||
|
|
||||||
|
// Union bitmaps for all matching dimension values in range.
|
||||||
|
// Use lazy iterator to allow unioning bitmaps one by one and avoid materializing all of them at once.
|
||||||
|
return selector.getBitmapFactory().union(
|
||||||
|
new Iterable<ImmutableBitmap>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Iterator<ImmutableBitmap> iterator()
|
||||||
|
{
|
||||||
|
return new Iterator<ImmutableBitmap>()
|
||||||
|
{
|
||||||
|
int currIndex = startIndex;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasNext()
|
||||||
|
{
|
||||||
|
return currIndex < endIndex;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ImmutableBitmap next()
|
||||||
|
{
|
||||||
|
while (currIndex < endIndex && !likeMatcher.matchesSuffixOnly(dimValues.get(currIndex))) {
|
||||||
|
currIndex++;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (currIndex == endIndex) {
|
||||||
|
return bitmapIndex.getBitmapFactory().makeEmptyImmutableBitmap();
|
||||||
|
}
|
||||||
|
|
||||||
|
return bitmapIndex.getBitmap(currIndex++);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void remove()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
// fallback
|
||||||
|
return Filters.matchPredicate(
|
||||||
|
dimension,
|
||||||
|
selector,
|
||||||
|
likeMatcher.predicateFactory(extractionFn).makeStringPredicate()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
||||||
|
{
|
||||||
|
return factory.makeValueMatcher(dimension, likeMatcher.predicateFactory(extractionFn));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
|
||||||
|
{
|
||||||
|
return selector.getBitmapIndex(dimension) != null;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,63 @@
|
||||||
|
/*
|
||||||
|
* 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 io.druid.jackson.DefaultObjectMapper;
|
||||||
|
import io.druid.query.extraction.SubstringDimExtractionFn;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
public class LikeDimFilterTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void testSerde() throws IOException
|
||||||
|
{
|
||||||
|
final ObjectMapper objectMapper = new DefaultObjectMapper();
|
||||||
|
final DimFilter filter = new LikeDimFilter("foo", "bar%", "@", new SubstringDimExtractionFn(1, 2));
|
||||||
|
final DimFilter filter2 = objectMapper.readValue(objectMapper.writeValueAsString(filter), DimFilter.class);
|
||||||
|
Assert.assertEquals(filter, filter2);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetCacheKey()
|
||||||
|
{
|
||||||
|
final DimFilter filter = new LikeDimFilter("foo", "bar%", "@", new SubstringDimExtractionFn(1, 2));
|
||||||
|
final DimFilter filter2 = new LikeDimFilter("foo", "bar%", "@", new SubstringDimExtractionFn(1, 2));
|
||||||
|
final DimFilter filter3 = new LikeDimFilter("foo", "bar%", null, new SubstringDimExtractionFn(1, 2));
|
||||||
|
Assert.assertArrayEquals(filter.getCacheKey(), filter2.getCacheKey());
|
||||||
|
Assert.assertFalse(Arrays.equals(filter.getCacheKey(), filter3.getCacheKey()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testEqualsAndHashCode()
|
||||||
|
{
|
||||||
|
final DimFilter filter = new LikeDimFilter("foo", "bar%", "@", new SubstringDimExtractionFn(1, 2));
|
||||||
|
final DimFilter filter2 = new LikeDimFilter("foo", "bar%", "@", new SubstringDimExtractionFn(1, 2));
|
||||||
|
final DimFilter filter3 = new LikeDimFilter("foo", "bar%", null, new SubstringDimExtractionFn(1, 2));
|
||||||
|
Assert.assertEquals(filter, filter2);
|
||||||
|
Assert.assertNotEquals(filter, filter3);
|
||||||
|
Assert.assertEquals(filter.hashCode(), filter2.hashCode());
|
||||||
|
Assert.assertNotEquals(filter.hashCode(), filter3.hashCode());
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,219 @@
|
||||||
|
/*
|
||||||
|
* 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.extraction.SubstringDimExtractionFn;
|
||||||
|
import io.druid.query.filter.DimFilter;
|
||||||
|
import io.druid.query.filter.LikeDimFilter;
|
||||||
|
import io.druid.segment.IndexBuilder;
|
||||||
|
import io.druid.segment.StorageAdapter;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
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 LikeFilterTest 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(null, null, null)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "0", "dim1", "")),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "1", "dim1", "foo")),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "2", "dim1", "foobar")),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "3", "dim1", "bar")),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "4", "dim1", "foobarbaz")),
|
||||||
|
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "5", "dim1", "foo%bar"))
|
||||||
|
);
|
||||||
|
|
||||||
|
public LikeFilterTest(
|
||||||
|
String testName,
|
||||||
|
IndexBuilder indexBuilder,
|
||||||
|
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||||
|
boolean optimize
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void tearDown() throws Exception
|
||||||
|
{
|
||||||
|
BaseFilterTest.tearDown(LikeFilterTest.class.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testExactMatch()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "bar", null, null),
|
||||||
|
ImmutableList.of("3")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testExactMatchWithEscape()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "@bar", "@", null),
|
||||||
|
ImmutableList.of("3")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testExactMatchWithExtractionFn()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "bar", null, new SubstringDimExtractionFn(3, 3)),
|
||||||
|
ImmutableList.of("2", "4")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPrefixMatch()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "foo%", null, null),
|
||||||
|
ImmutableList.of("1", "2", "4", "5")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPrefixMatchWithEscape()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "foo@%%", "@", null),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPrefixMatchWithExtractionFn()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "a%", null, new SubstringDimExtractionFn(1, null)),
|
||||||
|
ImmutableList.of("3")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWildcardMatch()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "%oba%", null, null),
|
||||||
|
ImmutableList.of("2", "4")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMatchEmptyString()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "", null, null),
|
||||||
|
ImmutableList.of("0")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMatchEmptyStringWithExtractionFn()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "", null, new SubstringDimExtractionFn(100, 1)),
|
||||||
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWildcardMatchWithEscape()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "%@%ba%", "@", null),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWildcardMatchEverything()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "%", "@", null),
|
||||||
|
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPrefixAndSuffixMatch()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "f%r", null, null),
|
||||||
|
ImmutableList.of("2", "5")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUnderscoreMatch()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "f_o", null, null),
|
||||||
|
ImmutableList.of("1")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSuffixMatchWithExtractionFn()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new LikeDimFilter("dim1", "%ar", null, new SubstringDimExtractionFn(3, 3)),
|
||||||
|
ImmutableList.of("2", "4")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
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));
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue