diff --git a/benchmarks/src/main/java/io/druid/benchmark/LikeFilterBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/LikeFilterBenchmark.java new file mode 100644 index 00000000000..def4f51f50e --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/LikeFilterBenchmark.java @@ -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 ints = generateInts(); + final GenericIndexed dictionary = GenericIndexed.fromIterable( + FluentIterable.from(ints) + .transform( + new Function() + { + @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() + { + @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 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 generateInts() + { + final List ints = new ArrayList<>(cardinality); + + for (int i = 0; i < cardinality; i++) { + ints.add(START_INT + step * i); + } + + return ints; + } +} diff --git a/docs/content/querying/filters.md b/docs/content/querying/filters.md index 7ea14d09b6b..c0757838e03 100644 --- a/docs/content/querying/filters.md +++ b/docs/content/querying/filters.md @@ -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. +### 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 -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?| |--------|-----------|---------|---------| @@ -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"| |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`: ```json diff --git a/processing/src/main/java/io/druid/query/filter/DimFilter.java b/processing/src/main/java/io/druid/query/filter/DimFilter.java index c6a726bcbde..4de0c061fcc 100644 --- a/processing/src/main/java/io/druid/query/filter/DimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/DimFilter.java @@ -38,7 +38,8 @@ import com.google.common.collect.RangeSet; @JsonSubTypes.Type(name="spatial", value=SpatialDimFilter.class), @JsonSubTypes.Type(name="in", value=InDimFilter.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 { diff --git a/processing/src/main/java/io/druid/query/filter/DimFilterUtils.java b/processing/src/main/java/io/druid/query/filter/DimFilterUtils.java index 6bb8492a4a8..2ca189f8702 100644 --- a/processing/src/main/java/io/druid/query/filter/DimFilterUtils.java +++ b/processing/src/main/java/io/druid/query/filter/DimFilterUtils.java @@ -49,6 +49,7 @@ public class DimFilterUtils static final byte IN_CACHE_ID = 0x9; static final byte BOUND_CACHE_ID = 0xA; static final byte INTERVAL_CACHE_ID = 0xB; + static final byte LIKE_CACHE_ID = 0xC; public static final byte STRING_SEPARATOR = (byte) 0xFF; static byte[] computeCacheKey(byte cacheIdKey, List filters) diff --git a/processing/src/main/java/io/druid/query/filter/LikeDimFilter.java b/processing/src/main/java/io/druid/query/filter/LikeDimFilter.java new file mode 100644 index 00000000000..eee92f5c96e --- /dev/null +++ b/processing/src/main/java/io/druid/query/filter/LikeDimFilter.java @@ -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 makeStringPredicate() + { + if (extractionFn != null) { + return new Predicate() + { + @Override + public boolean apply(String input) + { + return matches(extractionFn.apply(input)); + } + }; + } else { + return new Predicate() + { + @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 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(); + } +} diff --git a/processing/src/main/java/io/druid/segment/filter/BoundFilter.java b/processing/src/main/java/io/druid/segment/filter/BoundFilter.java index 683dea00d8a..946050f3f32 100644 --- a/processing/src/main/java/io/druid/segment/filter/BoundFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/BoundFilter.java @@ -61,12 +61,7 @@ public class BoundFilter implements Filter final BitmapIndex bitmapIndex = selector.getBitmapIndex(boundDimFilter.getDimension()); if (bitmapIndex == null || bitmapIndex.getCardinality() == 0) { - if (doesMatch(null)) { - return selector.getBitmapFactory() - .complement(selector.getBitmapFactory().makeEmptyImmutableBitmap(), selector.getNumRows()); - } else { - return selector.getBitmapFactory().makeEmptyImmutableBitmap(); - } + return doesMatch(null) ? Filters.allTrue(selector) : Filters.allFalse(selector); } // search for start, end indexes in the bitmaps; then include all bitmaps between those points diff --git a/processing/src/main/java/io/druid/segment/filter/Filters.java b/processing/src/main/java/io/druid/segment/filter/Filters.java index d35bae5e97b..e322480592b 100644 --- a/processing/src/main/java/io/druid/segment/filter/Filters.java +++ b/processing/src/main/java/io/druid/segment/filter/Filters.java @@ -87,6 +87,17 @@ public class Filters 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. * diff --git a/processing/src/main/java/io/druid/segment/filter/LikeFilter.java b/processing/src/main/java/io/druid/segment/filter/LikeFilter.java new file mode 100644 index 00000000000..7814eb72453 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/filter/LikeFilter.java @@ -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 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() + { + @Override + public Iterator iterator() + { + return new Iterator() + { + 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; + } +} diff --git a/processing/src/test/java/io/druid/query/filter/LikeDimFilterTest.java b/processing/src/test/java/io/druid/query/filter/LikeDimFilterTest.java new file mode 100644 index 00000000000..6defa1cb5db --- /dev/null +++ b/processing/src/test/java/io/druid/query/filter/LikeDimFilterTest.java @@ -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()); + } +} diff --git a/processing/src/test/java/io/druid/segment/filter/LikeFilterTest.java b/processing/src/test/java/io/druid/segment/filter/LikeFilterTest.java new file mode 100644 index 00000000000..00fc45b7382 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/filter/LikeFilterTest.java @@ -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> PARSER = new MapInputRowParser( + new TimeAndDimsParseSpec( + new TimestampSpec(TIMESTAMP_COLUMN, "iso", new DateTime("2000")), + new DimensionsSpec(null, null, null) + ) + ); + + private static final List ROWS = ImmutableList.of( + PARSER.parse(ImmutableMap.of("dim0", "0", "dim1", "")), + PARSER.parse(ImmutableMap.of("dim0", "1", "dim1", "foo")), + PARSER.parse(ImmutableMap.of("dim0", "2", "dim1", "foobar")), + PARSER.parse(ImmutableMap.of("dim0", "3", "dim1", "bar")), + PARSER.parse(ImmutableMap.of("dim0", "4", "dim1", "foobarbaz")), + PARSER.parse(ImmutableMap.of("dim0", "5", "dim1", "foo%bar")) + ); + + public LikeFilterTest( + String testName, + IndexBuilder indexBuilder, + Function> 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 expectedRows + ) + { + Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0")); + Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter)); + } +}