diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java index cb630a2e400b..c084e8c3b6e2 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -48,9 +48,8 @@ import org.apache.druid.segment.join.JoinTestHelper; import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.join.JoinableClause; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; -import org.apache.druid.segment.join.filter.JoinableClauses; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -140,19 +139,20 @@ public void setup() throws IOException ) ) ); - JoinFilterPreAnalysis preAnalysisLookupStringKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClausesLookupStringKey), - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + JoinFilterPreAnalysisGroup preAnalysisGroupLookupStringKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0, + false + ) ); + hashJoinLookupStringKeySegment = new HashJoinSegment( baseSegment, joinableClausesLookupStringKey, - preAnalysisLookupStringKey + preAnalysisGroupLookupStringKey ); List joinableClausesLookupLongKey = ImmutableList.of( @@ -167,19 +167,20 @@ public void setup() throws IOException ) ) ); - JoinFilterPreAnalysis preAnalysisLookupLongKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClausesLookupLongKey), - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + + JoinFilterPreAnalysisGroup preAnalysisGroupLookupLongKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0, + false + ) ); hashJoinLookupLongKeySegment = new HashJoinSegment( baseSegment, joinableClausesLookupLongKey, - preAnalysisLookupLongKey + preAnalysisGroupLookupLongKey ); List joinableClausesIndexedTableStringKey = ImmutableList.of( @@ -194,19 +195,20 @@ public void setup() throws IOException ) ) ); - JoinFilterPreAnalysis preAnalysisIndexedTableStringKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClausesIndexedTableStringKey), - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + + JoinFilterPreAnalysisGroup preAnalysisGroupIndexedStringKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0, + false + ) ); hashJoinIndexedTableStringKeySegment = new HashJoinSegment( baseSegment, joinableClausesIndexedTableStringKey, - preAnalysisIndexedTableStringKey + preAnalysisGroupIndexedStringKey ); List joinableClausesIndexedTableLonggKey = ImmutableList.of( @@ -221,19 +223,19 @@ public void setup() throws IOException ) ) ); - JoinFilterPreAnalysis preAnalysisIndexedTableLongKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClausesIndexedTableLonggKey), - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + JoinFilterPreAnalysisGroup preAnalysisGroupIndexedLongKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0, + false + ) ); hashJoinIndexedTableLongKeySegment = new HashJoinSegment( baseSegment, joinableClausesIndexedTableLonggKey, - preAnalysisIndexedTableLongKey + preAnalysisGroupIndexedLongKey ); final Map countryCodeToNameMap = JoinTestHelper.createCountryIsoCodeToNameLookup().getMap(); diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index b5a7be0b7dd8..1c35004063e0 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -52,6 +52,7 @@ public class QueryContexts public static final String JOIN_FILTER_REWRITE_ENABLE_KEY = "enableJoinFilterRewrite"; public static final String JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY = "enableJoinFilterRewriteValueColumnFilters"; public static final String JOIN_FILTER_REWRITE_MAX_SIZE_KEY = "joinFilterRewriteMaxSize"; + public static final String JOIN_FILTER_REWRITE_USE_OLD_REWRITE_MODE = "joinFilterRewriteUseOldRewriteMode"; public static final String USE_FILTER_CNF_KEY = "useFilterCNF"; public static final boolean DEFAULT_BY_SEGMENT = false; @@ -68,6 +69,7 @@ public class QueryContexts public static final boolean DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN = true; public static final boolean DEFAULT_ENABLE_JOIN_FILTER_REWRITE = true; public static final boolean DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS = false; + public static final boolean DEFAULT_JOIN_FILTER_REWRITE_USE_OLD_REWRITE_MODE = false; public static final long DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE = 10000; public static final boolean DEFAULT_USE_FILTER_CNF = false; @@ -264,6 +266,26 @@ public static boolean getEnableJoinFilterRewrite(Query query) return parseBoolean(query, JOIN_FILTER_REWRITE_ENABLE_KEY, DEFAULT_ENABLE_JOIN_FILTER_REWRITE); } + /** + * This is an undocumented option provided as a transition tool: + * + * The join filter rewrites originally performed the pre-analysis phase prior to any per-segment processing, + * analyzing only the filter in the top-level of the query. + * + * This did not work for nested queries (see https://github.com/apache/druid/pull/9978), so the rewrite pre-analysis + * was moved into the cursor creation of the {@link org.apache.druid.segment.join.HashJoinSegmentStorageAdapter}. + * This design requires synchronization across multiple segment processing threads; the old rewrite mode + * is kept temporarily available in case issues arise with the new mode, and the user does not run queries with the + * affected nested shape. + */ + public static boolean getUseJoinFilterRewriteOldRewriteMode(Query query) + { + return parseBoolean( + query, + JOIN_FILTER_REWRITE_USE_OLD_REWRITE_MODE, + DEFAULT_JOIN_FILTER_REWRITE_USE_OLD_REWRITE_MODE + ); + } public static Query withMaxScatterGatherBytes(Query query, long maxScatterGatherBytesLimit) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/AndDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/AndDimFilter.java index 18df9d67a170..19e13db0fe29 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/AndDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/AndDimFilter.java @@ -20,12 +20,14 @@ package org.apache.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; 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 com.google.common.collect.TreeRangeSet; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.segment.filter.AndFilter; import org.apache.druid.segment.filter.Filters; @@ -43,6 +45,9 @@ public class AndDimFilter implements DimFilter private final List fields; + @JsonIgnore + private Integer fieldsHashCode; + @JsonCreator public AndDimFilter( @JsonProperty("fields") List fields @@ -67,7 +72,7 @@ public List getFields() @Override public byte[] getCacheKey() { - return DimFilterUtils.computeCacheKey(DimFilterUtils.AND_CACHE_ID, fields); + return new CacheKeyBuilder(DimFilterUtils.AND_CACHE_ID).appendInt(hashCode()).build(); } @Override @@ -142,7 +147,10 @@ public boolean equals(Object o) @Override public int hashCode() { - return fields != null ? fields.hashCode() : 0; + if (fieldsHashCode == null) { + fieldsHashCode = fields != null ? fields.hashCode() : 0; + } + return fieldsHashCode; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java index 3fcd719e25f5..f8e018eda9ca 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java @@ -23,7 +23,6 @@ import com.google.common.collect.RangeSet; import org.apache.druid.timeline.partition.ShardSpec; -import java.nio.ByteBuffer; import java.util.HashMap; import java.util.LinkedHashSet; import java.util.List; @@ -57,29 +56,6 @@ public class DimFilterUtils public static final byte STRING_SEPARATOR = (byte) 0xFF; - static byte[] computeCacheKey(byte cacheIdKey, List filters) - { - if (filters.size() == 1) { - return filters.get(0).getCacheKey(); - } - - byte[][] cacheKeys = new byte[filters.size()][]; - int totalSize = 0; - int index = 0; - for (DimFilter field : filters) { - cacheKeys[index] = field.getCacheKey(); - totalSize += cacheKeys[index].length; - ++index; - } - - ByteBuffer retVal = ByteBuffer.allocate(1 + totalSize); - retVal.put(cacheIdKey); - for (byte[] cacheKey : cacheKeys) { - retVal.put(cacheKey); - } - return retVal.array(); - } - /** * Filter the given iterable of objects by removing any object whose ShardSpec, obtained from the converter function, * does not fit in the RangeSet of the dimFilter {@link DimFilter#getDimensionRangeSet(String)}. The returned set diff --git a/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java index d5ec5588ba38..4fb3799ac291 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java @@ -81,6 +81,9 @@ public class InDimFilter implements DimFilter @JsonIgnore private byte[] cacheKey; + @JsonIgnore + private final Supplier valuesHashCode; + @JsonCreator public InDimFilter( @JsonProperty("dimension") String dimension, @@ -107,6 +110,7 @@ public InDimFilter( this.longPredicateSupplier = getLongPredicateSupplier(); this.floatPredicateSupplier = getFloatPredicateSupplier(); this.doublePredicateSupplier = getDoublePredicateSupplier(); + this.valuesHashCode = Suppliers.memoize(() -> computeValuesHashCode(values)); } /** @@ -150,22 +154,12 @@ public FilterTuning getFilterTuning() public byte[] getCacheKey() { if (cacheKey == null) { - final List sortedValues = new ArrayList<>(values); - sortedValues.sort(Comparator.nullsFirst(Ordering.natural())); - final Hasher hasher = Hashing.sha256().newHasher(); - for (String v : sortedValues) { - if (v == null) { - hasher.putInt(0); - } else { - hasher.putString(v, StandardCharsets.UTF_8); - } - } cacheKey = new CacheKeyBuilder(DimFilterUtils.IN_CACHE_ID) .appendString(dimension) .appendByte(DimFilterUtils.STRING_SEPARATOR) .appendByteArray(extractionFn == null ? new byte[0] : extractionFn.getCacheKey()) .appendByte(DimFilterUtils.STRING_SEPARATOR) - .appendByteArray(hasher.hash().asBytes()) + .appendInt(valuesHashCode.get()) .build(); } return cacheKey; @@ -237,7 +231,8 @@ public Filter toFilter() floatPredicateSupplier, doublePredicateSupplier, extractionFn, - filterTuning + filterTuning, + valuesHashCode ); } @@ -290,8 +285,9 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } + InDimFilter that = (InDimFilter) o; - return values.equals(that.values) && + return valuesHashCode.get().equals(that.valuesHashCode.get()) && dimension.equals(that.dimension) && Objects.equals(extractionFn, that.extractionFn) && Objects.equals(filterTuning, that.filterTuning); @@ -300,7 +296,22 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(values, dimension, extractionFn, filterTuning); + return Objects.hash(valuesHashCode.get(), dimension, extractionFn, filterTuning); + } + + public static int computeValuesHashCode(Set values) + { + final List sortedValues = new ArrayList<>(values); + sortedValues.sort(Comparator.nullsFirst(Ordering.natural())); + final Hasher hasher = Hashing.sha256().newHasher(); + for (String v : sortedValues) { + if (v == null) { + hasher.putInt(0); + } else { + hasher.putString(v, StandardCharsets.UTF_8); + } + } + return hasher.hash().asInt(); } private DruidLongPredicate createLongPredicate() diff --git a/processing/src/main/java/org/apache/druid/query/filter/OrDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/OrDimFilter.java index ca8c105b7a3d..01d7e66fafc1 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/OrDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/OrDimFilter.java @@ -20,12 +20,14 @@ package org.apache.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; 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 com.google.common.collect.TreeRangeSet; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.OrFilter; @@ -44,6 +46,9 @@ public class OrDimFilter implements DimFilter private final List fields; + @JsonIgnore + private Integer fieldsHashCode; + @JsonCreator public OrDimFilter(@JsonProperty("fields") List fields) { @@ -75,7 +80,7 @@ public List getFields() @Override public byte[] getCacheKey() { - return DimFilterUtils.computeCacheKey(DimFilterUtils.OR_CACHE_ID, fields); + return new CacheKeyBuilder(DimFilterUtils.OR_CACHE_ID).appendInt(hashCode()).build(); } @Override @@ -148,7 +153,10 @@ public boolean equals(Object o) @Override public int hashCode() { - return fields != null ? fields.hashCode() : 0; + if (fieldsHashCode == null) { + fieldsHashCode = fields != null ? fields.hashCode() : 0; + } + return fieldsHashCode; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java index 7bd2a847c081..462644425dc1 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java @@ -249,8 +249,8 @@ private List> verifySubtotalsSpec( return subtotalsSpec; } - @Override @JsonProperty + @Override public VirtualColumns getVirtualColumns() { return virtualColumns; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 459b8326d217..be5d24b71b34 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -183,8 +183,8 @@ private Integer validateAndGetMaxSegmentPartitionsOrderedInMemory() return maxSegmentPartitionsOrderedInMemory; } - @Override @JsonProperty + @Override public VirtualColumns getVirtualColumns() { return virtualColumns; diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java index 729d839d0298..94c187346da5 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -103,8 +103,8 @@ public String getType() return Query.TIMESERIES; } - @Override @JsonProperty + @Override public VirtualColumns getVirtualColumns() { return virtualColumns; diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 3c301e584af8..f5bdec1982a0 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -113,8 +113,8 @@ public String getType() return TOPN; } - @Override @JsonProperty + @Override public VirtualColumns getVirtualColumns() { return virtualColumns; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java index 696bd8b5c9c9..fa663141ea34 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java @@ -22,6 +22,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.druid.collections.bitmap.ImmutableBitmap; @@ -39,6 +41,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; +import javax.annotation.concurrent.Immutable; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -47,12 +50,15 @@ /** */ +@Immutable public class AndFilter implements BooleanFilter { private static final Joiner AND_JOINER = Joiner.on(" && "); private final Set filters; + private final Supplier filtersHashCode; + @VisibleForTesting public AndFilter(List filters) { @@ -63,6 +69,7 @@ public AndFilter(Set filters) { Preconditions.checkArgument(filters.size() > 0, "Can't construct empty AndFilter"); this.filters = filters; + this.filtersHashCode = Suppliers.memoize(() -> Objects.hash(getFilters())); } public static ImmutableBitmap getBitmapIndex( @@ -257,12 +264,12 @@ public boolean equals(Object o) return false; } AndFilter andFilter = (AndFilter) o; - return Objects.equals(getFilters(), andFilter.getFilters()); + return Objects.equals(hashCode(), andFilter.hashCode()); } @Override public int hashCode() { - return Objects.hash(getFilters()); + return filtersHashCode.get(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/InFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/InFilter.java index dfc4d711d4ba..8b8ce359d61e 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/InFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/InFilter.java @@ -22,7 +22,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicate; import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableSet; +import com.google.errorprone.annotations.Immutable; import it.unimi.dsi.fastutil.ints.IntIterable; import it.unimi.dsi.fastutil.ints.IntIterator; import org.apache.druid.collections.bitmap.ImmutableBitmap; @@ -36,6 +38,7 @@ import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; +import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; @@ -63,6 +66,7 @@ * In default null handling mode, this filter is equivalent to {@code (dimension IN [values])} or * {@code (dimension IN [non-null values, ''])} when {@link #values} contains nulls. */ +@Immutable public class InFilter implements Filter { private final String dimension; @@ -72,6 +76,7 @@ public class InFilter implements Filter private final Supplier longPredicateSupplier; private final Supplier floatPredicateSupplier; private final Supplier doublePredicateSupplier; + private final Supplier valuesHashCode; public InFilter( String dimension, @@ -80,7 +85,8 @@ public InFilter( Supplier floatPredicateSupplier, Supplier doublePredicateSupplier, ExtractionFn extractionFn, - FilterTuning filterTuning + FilterTuning filterTuning, + Supplier valuesHashCode ) { this.dimension = dimension; @@ -90,6 +96,9 @@ public InFilter( this.longPredicateSupplier = longPredicateSupplier; this.floatPredicateSupplier = floatPredicateSupplier; this.doublePredicateSupplier = doublePredicateSupplier; + this.valuesHashCode = valuesHashCode == null + ? Suppliers.memoize(() -> InDimFilter.computeValuesHashCode(values)) + : valuesHashCode; } @Override @@ -201,7 +210,8 @@ public Filter rewriteRequiredColumns(Map columnRewrites) floatPredicateSupplier, doublePredicateSupplier, extractionFn, - filterTuning + filterTuning, + valuesHashCode ); } @@ -225,7 +235,13 @@ public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, Bitm private DruidPredicateFactory getPredicateFactory() { - return new InFilterDruidPredicateFactory(extractionFn, values, longPredicateSupplier, floatPredicateSupplier, doublePredicateSupplier); + return new InFilterDruidPredicateFactory( + extractionFn, + values, + longPredicateSupplier, + floatPredicateSupplier, + doublePredicateSupplier + ); } @Override @@ -239,7 +255,7 @@ public boolean equals(Object o) } InFilter inFilter = (InFilter) o; return Objects.equals(dimension, inFilter.dimension) && - Objects.equals(values, inFilter.values) && + Objects.equals(valuesHashCode.get(), inFilter.valuesHashCode.get()) && Objects.equals(extractionFn, inFilter.extractionFn) && Objects.equals(filterTuning, inFilter.filterTuning); } @@ -247,7 +263,7 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(dimension, values, extractionFn, filterTuning); + return Objects.hash(dimension, valuesHashCode.get(), extractionFn, filterTuning); } @VisibleForTesting diff --git a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java index 46cd9e246db4..bd32cc2527c8 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java @@ -22,6 +22,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.Iterables; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.java.util.common.StringUtils; @@ -39,6 +41,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; +import javax.annotation.concurrent.Immutable; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -47,12 +50,15 @@ /** */ +@Immutable public class OrFilter implements BooleanFilter { private static final Joiner OR_JOINER = Joiner.on(" || "); private final Set filters; + private final Supplier filtersHashCode; + @VisibleForTesting public OrFilter(List filters) { @@ -64,6 +70,7 @@ public OrFilter(Set filters) Preconditions.checkArgument(filters.size() > 0, "Can't construct empty OrFilter (the universe does not exist)"); this.filters = filters; + this.filtersHashCode = Suppliers.memoize(() -> Objects.hash(getFilters())); } @Override @@ -242,12 +249,12 @@ public boolean equals(Object o) return false; } OrFilter orFilter = (OrFilter) o; - return Objects.equals(getFilters(), orFilter.getFilters()); + return Objects.equals(hashCode(), orFilter.hashCode()); } @Override public int hashCode() { - return Objects.hash(getFilters()); + return filtersHashCode.get(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java index d321af958008..82640911f72b 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java @@ -24,7 +24,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -41,23 +41,24 @@ public class HashJoinSegment extends AbstractSegment { private final Segment baseSegment; private final List clauses; - private final JoinFilterPreAnalysis joinFilterPreAnalysis; + private final JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup; /** * @param baseSegment The left-hand side base segment * @param clauses The right-hand side clauses. The caller is responsible for ensuring that there are no * duplicate prefixes or prefixes that shadow each other across the clauses - * @param joinFilterPreAnalysis Pre-analysis computed by {@link org.apache.druid.segment.join.filter.JoinFilterAnalyzer#computeJoinFilterPreAnalysis} + * @param joinFilterPreAnalysisGroup Pre-analysis group that holds all of the JoinFilterPreAnalysis results within + * the scope of a query */ public HashJoinSegment( Segment baseSegment, List clauses, - JoinFilterPreAnalysis joinFilterPreAnalysis + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup ) { this.baseSegment = baseSegment; this.clauses = clauses; - this.joinFilterPreAnalysis = joinFilterPreAnalysis; + this.joinFilterPreAnalysisGroup = joinFilterPreAnalysisGroup; // Verify 'clauses' is nonempty (otherwise it's a waste to create this object, and the caller should know) if (clauses.isEmpty()) { @@ -90,7 +91,7 @@ public QueryableIndex asQueryableIndex() @Override public StorageAdapter asStorageAdapter() { - return new HashJoinSegmentStorageAdapter(baseSegment.asStorageAdapter(), clauses, joinFilterPreAnalysis); + return new HashJoinSegmentStorageAdapter(baseSegment.asStorageAdapter(), clauses, joinFilterPreAnalysisGroup); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 65c25d498b56..7102bf64f448 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -21,7 +21,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -38,6 +37,7 @@ import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.segment.join.filter.JoinFilterSplit; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -47,7 +47,6 @@ import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; -import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -55,22 +54,22 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter { private final StorageAdapter baseAdapter; private final List clauses; - private final JoinFilterPreAnalysis joinFilterPreAnalysis; + private final JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup; /** * @param baseAdapter A StorageAdapter for the left-hand side base segment * @param clauses The right-hand side clauses. The caller is responsible for ensuring that there are no - * duplicate prefixes or prefixes that shadow each other across the clauses - */ + * @param joinFilterPreAnalysisGroup Pre-analysis group that holds all of the JoinFilterPreAnalysis results within + * the scope of a query */ HashJoinSegmentStorageAdapter( StorageAdapter baseAdapter, List clauses, - final JoinFilterPreAnalysis joinFilterPreAnalysis + final JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup ) { this.baseAdapter = baseAdapter; this.clauses = clauses; - this.joinFilterPreAnalysis = joinFilterPreAnalysis; + this.joinFilterPreAnalysisGroup = joinFilterPreAnalysisGroup; } @Override @@ -209,13 +208,17 @@ public Sequence makeCursors( @Nullable final QueryMetrics queryMetrics ) { - if (!Objects.equals(joinFilterPreAnalysis.getOriginalFilter(), filter)) { - throw new ISE( - "Filter provided to cursor [%s] does not match join pre-analysis filter [%s]", + JoinFilterPreAnalysis jfpa; + if (joinFilterPreAnalysisGroup.getJoinFilterRewriteConfig().isOldRewriteMode()) { + jfpa = joinFilterPreAnalysisGroup.getPreAnalysisForOldRewriteMode(); + } else { + jfpa = joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent( filter, - joinFilterPreAnalysis.getOriginalFilter() + clauses, + virtualColumns ); } + final List preJoinVirtualColumns = new ArrayList<>(); final List postJoinVirtualColumns = new ArrayList<>(); @@ -225,7 +228,7 @@ public Sequence makeCursors( postJoinVirtualColumns ); - JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(jfpa); preJoinVirtualColumns.addAll(joinFilterSplit.getPushDownVirtualColumns()); // Soon, we will need a way to push filters past a join when possible. This could potentially be done right here diff --git a/processing/src/main/java/org/apache/druid/segment/join/Joinables.java b/processing/src/main/java/org/apache/druid/segment/join/Joinables.java index 76bd19e877d2..c9c765484a1d 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/Joinables.java +++ b/processing/src/main/java/org/apache/druid/segment/join/Joinables.java @@ -25,9 +25,9 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.segment.join.filter.JoinableClauses; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; @@ -71,33 +71,23 @@ public static boolean isPrefixedBy(final String columnName, final String prefix) /** * Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join * clauses is > 0). If mapping is not needed, this method will return {@link Function#identity()}. - * - * @param clauses pre-joinable clauses - * @param joinableFactory factory for joinables - * @param cpuTimeAccumulator an accumulator that we will add CPU nanos to; this is part of the function to encourage - * callers to remember to track metrics on CPU time required for creation of Joinables - * @param enableFilterPushDown whether to enable filter push down optimizations to the base segment. In production - * this should generally be {@code QueryContexts.getEnableJoinFilterPushDown(query)}. - * @param enableFilterRewrite whether to enable filter rewrite optimizations for RHS columns. In production - * this should generally be {@code QueryContexts.getEnableJoinFilterRewrite(query)}. - * @param enableRewriteValueColumnFilters whether to enable filter rewrite optimizations for RHS columns that are not - * key columns. In production this should generally - * be {@code QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query)}. - * @param filterRewriteMaxSize the max allowed size of correlated value sets for RHS rewrites. In production - * this should generally be {@code QueryContexts.getJoinFilterRewriteMaxSize(query)}. - * @param originalFilter The original filter from the query. - * @param virtualColumns The virtual columns from the query. + * @param clauses pre-joinable clauses + * @param joinableFactory factory for joinables + * @param cpuTimeAccumulator an accumulator that we will add CPU nanos to; this is part of the function to encourage + * callers to remember to track metrics on CPU time required for creation of Joinables + * @param joinFilterRewriteConfig Configuration options for the join filter rewrites + * @param originalTopLevelFilterForOldRewriteMode The filter from the top level of the query. Only used if + * joinFilterRewriteConfig.isOldRewriteMode() is true. + * @param virtualColumnsForOldRewriteMode The virtual columns from the top level of the query. Only used if + * joinFilterRewriteConfig.isOldRewriteMode() is true. */ public static Function createSegmentMapFn( final List clauses, final JoinableFactory joinableFactory, final AtomicLong cpuTimeAccumulator, - final boolean enableFilterPushDown, - final boolean enableFilterRewrite, - final boolean enableRewriteValueColumnFilters, - final long filterRewriteMaxSize, - final Filter originalFilter, - final VirtualColumns virtualColumns + final JoinFilterRewriteConfig joinFilterRewriteConfig, + @Nullable final Filter originalTopLevelFilterForOldRewriteMode, + @Nullable final VirtualColumns virtualColumnsForOldRewriteMode ) { // compute column correlations here and RHS correlated values @@ -108,16 +98,18 @@ public static Function createSegmentMapFn( return Function.identity(); } else { final JoinableClauses joinableClauses = JoinableClauses.createClauses(clauses, joinableFactory); - JoinFilterPreAnalysis jfpa = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - virtualColumns, - originalFilter, - enableFilterPushDown, - enableFilterRewrite, - enableRewriteValueColumnFilters, - filterRewriteMaxSize - ); - return baseSegment -> new HashJoinSegment(baseSegment, joinableClauses.getJoinableClauses(), jfpa); + final JoinFilterPreAnalysisGroup jfpag = new JoinFilterPreAnalysisGroup(joinFilterRewriteConfig); + + // compatiblity mode + if (joinFilterRewriteConfig.isOldRewriteMode()) { + jfpag.performAnalysisForOldRewriteMode( + originalTopLevelFilterForOldRewriteMode, + joinableClauses.getJoinableClauses(), + virtualColumnsForOldRewriteMode + ); + } + + return baseSegment -> new HashJoinSegment(baseSegment, joinableClauses.getJoinableClauses(), jfpag); } } ); diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java index f4f7d31be1cf..06475ad89e51 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java @@ -32,6 +32,7 @@ import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.OrFilter; import org.apache.druid.segment.filter.SelectorFilter; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import java.util.ArrayList; @@ -85,23 +86,14 @@ public class JoinFilterAnalyzer * @param joinableClauses The joinable clauses from the query * @param virtualColumns The virtual columns from the query * @param originalFilter The original filter from the query - * @param enableFilterPushDown Whether to enable filter push down - * @param enableFilterRewrite Whether to enable rewrites of filters involving RHS columns - * @param enableRewriteValueColumnFilters Whether to enable rewrites of filters invovling RHS non-key columns - * @param filterRewriteMaxSize The maximum size of the correlated value set for rewritten filters. - * If the correlated value set size exceeds this, the filter will not be - * rewritten and pushed down. - * + * @param joinFilterRewriteConfig Configuration options for the join rewrites * @return A JoinFilterPreAnalysis containing information determined in this pre-analysis step. */ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( JoinableClauses joinableClauses, VirtualColumns virtualColumns, Filter originalFilter, - boolean enableFilterPushDown, - boolean enableFilterRewrite, - boolean enableRewriteValueColumnFilters, - long filterRewriteMaxSize + JoinFilterRewriteConfig joinFilterRewriteConfig ) { final List preJoinVirtualColumns = new ArrayList<>(); @@ -109,10 +101,8 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( joinableClauses.splitVirtualColumns(virtualColumns, preJoinVirtualColumns, postJoinVirtualColumns); JoinFilterPreAnalysis.Builder preAnalysisBuilder = - new JoinFilterPreAnalysis.Builder(joinableClauses, originalFilter, postJoinVirtualColumns) - .withEnableFilterPushDown(enableFilterPushDown) - .withEnableFilterRewrite(enableFilterRewrite); - if (originalFilter == null || !enableFilterPushDown) { + new JoinFilterPreAnalysis.Builder(joinFilterRewriteConfig, joinableClauses, originalFilter, postJoinVirtualColumns); + if (originalFilter == null || !joinFilterRewriteConfig.isEnableFilterPushDown()) { return preAnalysisBuilder.build(); } @@ -135,7 +125,7 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( preAnalysisBuilder .withNormalizedBaseTableClauses(normalizedBaseTableClauses) .withNormalizedJoinTableClauses(normalizedJoinTableClauses); - if (!enableFilterRewrite) { + if (!joinFilterRewriteConfig.isEnableFilterRewrite()) { return preAnalysisBuilder.build(); } @@ -146,8 +136,8 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( normalizedJoinTableClauses, equiconditions, joinableClauses, - enableRewriteValueColumnFilters, - filterRewriteMaxSize + joinFilterRewriteConfig.isEnableRewriteValueColumnFilters(), + joinFilterRewriteConfig.getFilterRewriteMaxSize() ); return preAnalysisBuilder.withCorrelations(correlations) diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java index ae3731db77a2..de842c3dde02 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java @@ -24,6 +24,7 @@ import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.join.Equality; import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -38,11 +39,11 @@ * A JoinFilterPreAnalysis contains filter push down/rewrite information that does not have per-segment dependencies. * This includes: * - The query's JoinableClauses list - * - The query's original filter (if any) + * - The original filter that an analysis was performed ons * - A list of filter clauses from the original filter's CNF representation that only reference the base table * - A list of filter clauses from the original filter's CNF representation that reference RHS join tables * - A list of virtual columns that can only be computed post-join - * - Control flag booleans for whether filter push down and RHS rewrites are enabled. + * - The JoinFilterRewriteConfig that this pre-analysis is associated with. */ public class JoinFilterPreAnalysis { @@ -51,10 +52,9 @@ public class JoinFilterPreAnalysis private final List normalizedBaseTableClauses; private final List normalizedJoinTableClauses; private final JoinFilterCorrelations correlations; - private final boolean enableFilterPushDown; - private final boolean enableFilterRewrite; private final List postJoinVirtualColumns; private final Equiconditions equiconditions; + private final JoinFilterRewriteConfig rewriteConfig; private JoinFilterPreAnalysis( final JoinableClauses joinableClauses, @@ -63,9 +63,8 @@ private JoinFilterPreAnalysis( final List normalizedBaseTableClauses, final List normalizedJoinTableClauses, JoinFilterCorrelations correlations, - final boolean enableFilterPushDown, - final boolean enableFilterRewrite, - final Equiconditions equiconditions + final Equiconditions equiconditions, + final JoinFilterRewriteConfig rewriteConfig ) { this.joinableClauses = joinableClauses; @@ -74,8 +73,7 @@ private JoinFilterPreAnalysis( this.normalizedBaseTableClauses = normalizedBaseTableClauses; this.normalizedJoinTableClauses = normalizedJoinTableClauses; this.correlations = correlations; - this.enableFilterPushDown = enableFilterPushDown; - this.enableFilterRewrite = enableFilterRewrite; + this.rewriteConfig = rewriteConfig; this.equiconditions = equiconditions; } @@ -116,12 +114,12 @@ public Map> getCorrelationsByD public boolean isEnableFilterPushDown() { - return enableFilterPushDown; + return rewriteConfig.isEnableFilterPushDown(); } public boolean isEnableFilterRewrite() { - return enableFilterRewrite; + return rewriteConfig.isEnableFilterRewrite(); } public Equiconditions getEquiconditions() @@ -134,22 +132,23 @@ public Equiconditions getEquiconditions() */ public static class Builder { + @Nonnull private final JoinFilterRewriteConfig rewriteConfig; @Nonnull private final JoinableClauses joinableClauses; @Nullable private final Filter originalFilter; @Nullable private List normalizedBaseTableClauses; @Nullable private List normalizedJoinTableClauses; @Nullable private JoinFilterCorrelations correlations; - private boolean enableFilterPushDown = false; - private boolean enableFilterRewrite = false; @Nonnull private final List postJoinVirtualColumns; @Nonnull private Equiconditions equiconditions = new Equiconditions(Collections.emptyMap()); public Builder( + @Nonnull JoinFilterRewriteConfig rewriteConfig, @Nonnull JoinableClauses joinableClauses, @Nullable Filter originalFilter, @Nonnull List postJoinVirtualColumns ) { + this.rewriteConfig = rewriteConfig; this.joinableClauses = joinableClauses; this.originalFilter = originalFilter; this.postJoinVirtualColumns = postJoinVirtualColumns; @@ -175,18 +174,6 @@ public Builder withCorrelations( return this; } - public Builder withEnableFilterPushDown(boolean enableFilterPushDown) - { - this.enableFilterPushDown = enableFilterPushDown; - return this; - } - - public Builder withEnableFilterRewrite(boolean enableFilterRewrite) - { - this.enableFilterRewrite = enableFilterRewrite; - return this; - } - public Equiconditions computeEquiconditionsFromJoinableClauses() { Map> equiconditionsMap = new HashMap<>(); @@ -212,9 +199,8 @@ public JoinFilterPreAnalysis build() normalizedBaseTableClauses, normalizedJoinTableClauses, correlations, - enableFilterPushDown, - enableFilterRewrite, - equiconditions + equiconditions, + rewriteConfig ); } diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterPreAnalysisGroup.java b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterPreAnalysisGroup.java new file mode 100644 index 000000000000..72804718822c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterPreAnalysisGroup.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.join.filter.rewrite; + +import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; +import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.join.filter.JoinableClauses; + +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A JoinFilterPreAnalysisGroup holds all of the JoinFilterPreAnalysis objects for a given query and + * also stores the per-query parameters that control the filter rewrite operations (from the query context). + * + * The analyses map is keyed by (Filter, JoinableClause list, VirtualColumns): each Filter in the map belongs to a + * separate level of query (e.g. outer query, subquery level 1, etc.) + * + * A concurrent hash map is used since the per-level Filter processing occurs across multiple threads. + */ +public class JoinFilterPreAnalysisGroup +{ + private final JoinFilterRewriteConfig joinFilterRewriteConfig; + private final ConcurrentHashMap analyses; + + /** + * This is an undocumented option provided as a transition tool: + * + * The join filter rewrites originally performed the pre-analysis phase prior to any per-segment processing, + * analyzing only the filter in the top-level of the query. + * + * This did not work for nested queries (see https://github.com/apache/druid/pull/9978), so the rewrite pre-analysis + * was moved into the cursor creation of the {@link org.apache.druid.segment.join.HashJoinSegmentStorageAdapter}. + * This design requires synchronization across multiple segment processing threads; the old rewrite mode + * is kept temporarily available in case issues arise with the new mode, and the user does not run queries with the + * affected nested shape. + */ + private JoinFilterPreAnalysis preAnalysisForOldRewriteMode; + + public JoinFilterPreAnalysisGroup( + JoinFilterRewriteConfig joinFilterRewriteConfig + ) + { + this.joinFilterRewriteConfig = joinFilterRewriteConfig; + this.analyses = new ConcurrentHashMap<>(); + } + + public JoinFilterRewriteConfig getJoinFilterRewriteConfig() + { + return joinFilterRewriteConfig; + } + + public JoinFilterPreAnalysis computeJoinFilterPreAnalysisIfAbsent( + Filter filter, + List clauses, + VirtualColumns virtualColumns + ) + { + JoinFilterPreAnalysisGroupKey key = new JoinFilterPreAnalysisGroupKey(filter, clauses, virtualColumns); + return analyses.computeIfAbsent( + key, + (groupKey) -> { + return JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinableClauses.fromList(clauses), + virtualColumns, + filter, + joinFilterRewriteConfig + ); + } + ); + } + + public JoinFilterPreAnalysis getAnalysis( + Filter filter, + List clauses, + VirtualColumns virtualColumns + ) + { + JoinFilterPreAnalysisGroupKey key = new JoinFilterPreAnalysisGroupKey(filter, clauses, virtualColumns); + return analyses.get(key); + } + + public void performAnalysisForOldRewriteMode( + Filter filter, + List clauses, + VirtualColumns virtualColumns + ) + { + preAnalysisForOldRewriteMode = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinableClauses.fromList(clauses), + virtualColumns, + filter, + joinFilterRewriteConfig + ); + } + + public JoinFilterPreAnalysis getPreAnalysisForOldRewriteMode() + { + return preAnalysisForOldRewriteMode; + } + + private static class JoinFilterPreAnalysisGroupKey + { + private final Filter filter; + private final List clauses; + private final VirtualColumns virtualColumns; + + public JoinFilterPreAnalysisGroupKey( + Filter filter, + List clauses, + VirtualColumns virtualColumns + ) + { + this.filter = filter; + this.clauses = clauses; + this.virtualColumns = virtualColumns; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JoinFilterPreAnalysisGroupKey that = (JoinFilterPreAnalysisGroupKey) o; + return Objects.equals(filter, that.filter) && + Objects.equals(clauses, that.clauses) && + Objects.equals(virtualColumns, that.virtualColumns); + } + + @Override + public int hashCode() + { + return Objects.hash(filter, clauses, virtualColumns); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterRewriteConfig.java b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterRewriteConfig.java new file mode 100644 index 000000000000..5f64d6a57ccf --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterRewriteConfig.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.join.filter.rewrite; + +/** + * A config class that holds properties that control how join filter rewrites behave. + */ +public class JoinFilterRewriteConfig +{ + /** + * Whether to enable filter push down optimizations to the base segment. + * In production this should generally be {@code QueryContexts.getEnableJoinFilterPushDown(query)}. + */ + private final boolean enableFilterPushDown; + + /** + * Whether to enable filter rewrite optimizations for RHS columns. + * In production this should generally be {@code QueryContexts.getEnableJoinFilterRewrite(query)}. + */ + private final boolean enableFilterRewrite; + + /** + * Whether to enable filter rewrite optimizations for RHS columns that are not key columns. + * In production this should generally be {@code QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query)}. + */ + private final boolean enableRewriteValueColumnFilters; + + /** + * The max allowed size of correlated value sets for RHS rewrites. In production + * This should generally be {@code QueryContexts.getJoinFilterRewriteMaxSize(query)}. + */ + private final long filterRewriteMaxSize; + + /** + * This is an undocumented option provided as a transition tool: + * + * The join filter rewrites originally performed the pre-analysis phase prior to any per-segment processing, + * analyzing only the filter in the top-level of the query. + * + * This did not work for nested queries (see https://github.com/apache/druid/pull/9978), so the rewrite pre-analysis + * was moved into the cursor creation of the {@link org.apache.druid.segment.join.HashJoinSegmentStorageAdapter}. + * This design requires synchronization across multiple segment processing threads; the old rewrite mode + * is kept temporarily available in case issues arise with the new mode, and the user does not run queries with the + * affected nested shape. + */ + private final boolean oldRewriteMode; + + + public JoinFilterRewriteConfig( + boolean enableFilterPushDown, + boolean enableFilterRewrite, + boolean enableRewriteValueColumnFilters, + long filterRewriteMaxSize, + boolean oldRewriteMode + ) + { + this.enableFilterPushDown = enableFilterPushDown; + this.enableFilterRewrite = enableFilterRewrite; + this.enableRewriteValueColumnFilters = enableRewriteValueColumnFilters; + this.filterRewriteMaxSize = filterRewriteMaxSize; + this.oldRewriteMode = oldRewriteMode; + } + + public boolean isEnableFilterPushDown() + { + return enableFilterPushDown; + } + + public boolean isEnableFilterRewrite() + { + return enableFilterRewrite; + } + + public boolean isEnableRewriteValueColumnFilters() + { + return enableRewriteValueColumnFilters; + } + + public long getFilterRewriteMaxSize() + { + return filterRewriteMaxSize; + } + + public boolean isOldRewriteMode() + { + return oldRewriteMode; + } +} diff --git a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java index 9cf5b8f67bdb..dfe9203c2703 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java @@ -20,9 +20,12 @@ package org.apache.druid.query.filter; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.common.config.NullHandling; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.query.extraction.RegexDimExtractionFn; @@ -144,4 +147,15 @@ public void testOptimizeSingleValueInToSelector() final InDimFilter filter = new InDimFilter("dim", Collections.singleton("v1"), null); Assert.assertEquals(new SelectorDimFilter("dim", "v1", null), filter.optimize()); } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(InDimFilter.class) + .usingGetClass() + .withNonnullFields("dimension", "values", "valuesHashCode") + .withIgnoredFields("values", "cacheKey", "longPredicateSupplier", "floatPredicateSupplier", "doublePredicateSupplier") + .withPrefabValues(Supplier.class, Suppliers.ofInstance(1), Suppliers.ofInstance(2)) + .verify(); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java index 3af7bf89ec1c..0efeb5800ba9 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java @@ -20,6 +20,8 @@ package org.apache.druid.segment.filter; import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; @@ -180,6 +182,15 @@ public void testNotAnd() @Test public void test_equals() { - EqualsVerifier.forClass(AndFilter.class).usingGetClass().withNonnullFields("filters").verify(); + EqualsVerifier.forClass(AndFilter.class) + .usingGetClass() + .withNonnullFields("filters", "filtersHashCode") + .withIgnoredFields("filters") + .withPrefabValues( + Supplier.class, + Suppliers.ofInstance(1), + Suppliers.ofInstance(2) + ) + .verify(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java index 6a0db8463feb..789be36e8e82 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java @@ -20,6 +20,8 @@ package org.apache.druid.segment.filter; import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -383,8 +385,9 @@ public void test_equals() { EqualsVerifier.forClass(InFilter.class) .usingGetClass() - .withNonnullFields("dimension", "values") - .withIgnoredFields("longPredicateSupplier", "floatPredicateSupplier", "doublePredicateSupplier") + .withNonnullFields("dimension", "values", "valuesHashCode") + .withIgnoredFields("values", "longPredicateSupplier", "floatPredicateSupplier", "doublePredicateSupplier") + .withPrefabValues(Supplier.class, Suppliers.ofInstance(1), Suppliers.ofInstance(2)) .verify(); } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java index 844a0895e941..65f234778ff9 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java @@ -19,14 +19,26 @@ package org.apache.druid.segment.filter; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Test; -public class OrFilterTest +public class OrFilterTest extends InitializedNullHandlingTest { @Test public void test_equals() { - EqualsVerifier.forClass(OrFilter.class).usingGetClass().withNonnullFields("filters").verify(); + EqualsVerifier.forClass(OrFilter.class) + .usingGetClass() + .withNonnullFields("filters", "filtersHashCode") + .withIgnoredFields("filters") + .withPrefabValues( + Supplier.class, + Suppliers.ofInstance(1), + Suppliers.ofInstance(2) + ) + .verify(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java index 45befeba36ed..402047576bf0 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java @@ -26,10 +26,8 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; -import org.apache.druid.segment.join.filter.JoinableClauses; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.segment.join.table.IndexedTableJoinable; @@ -47,6 +45,14 @@ public class BaseHashJoinSegmentStorageAdapterTest { + public static JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, + false + ); + public static final String FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX = "c1."; public static final String FACT_TO_COUNTRY_ON_NUMBER_PREFIX = "c2."; public static final String FACT_TO_REGION_PREFIX = "r1."; @@ -187,20 +193,12 @@ protected JoinableClause regionToCountry(final JoinType joinType) protected HashJoinSegmentStorageAdapter makeFactToCountrySegment() { - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT))), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); return new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)), - preAnalysis + joinFilterPreAnalysisGroup ); } @@ -222,4 +220,11 @@ protected void compareExpressionVirtualColumns( actualVirtualColumn.getParsedExpression().get().toString() ); } + + protected static JoinFilterPreAnalysisGroup makeDefaultConfigPreAnalysisGroup() + { + return new JoinFilterPreAnalysisGroup( + DEFAULT_JOIN_FILTER_REWRITE_CONFIG + ); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java index 2e863b2487a2..d65dbce7bf37 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java @@ -23,12 +23,10 @@ import com.google.common.collect.Lists; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.query.QueryContexts; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.OrDimFilter; @@ -37,9 +35,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.filter.SelectorFilter; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; -import org.apache.druid.segment.join.filter.JoinableClauses; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -302,21 +298,13 @@ public void test_makeCursors_factToCountryLeft() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -371,21 +359,13 @@ public void test_makeCursors_factToCountryLeftUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -438,21 +418,13 @@ public void test_makeCursors_factToCountryLeftUsingLookup() public void test_makeCursors_factToCountryInner() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.INNER)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -500,21 +472,13 @@ public void test_makeCursors_factToCountryInner() public void test_makeCursors_factToCountryInnerUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.INNER)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -564,21 +528,13 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumber() // is interpreted as 0 (a.k.a. Australia). List joinableClauses = ImmutableList.of(factToCountryOnNumber(JoinType.INNER)); Filter filter = new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -634,21 +590,13 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() // is interpreted as 0 (a.k.a. Australia). List joinableClauses = ImmutableList.of(factToCountryNameUsingNumberLookup(JoinType.INNER)); Filter filter = new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -700,21 +648,13 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFacts() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -741,21 +681,13 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFactsUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -781,21 +713,13 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -824,21 +748,13 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNullUsingLookup { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.RIGHT)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -866,21 +782,13 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -909,21 +817,13 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNullUsingLookup( { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.FULL)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -956,21 +856,13 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinable() null ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1003,21 +895,13 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinableUsingLookup() null ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1050,21 +934,13 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber", "10", null) ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1097,21 +973,13 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Norway", null) ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1157,21 +1025,13 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit ExprMacroTable.nil() ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1238,22 +1098,13 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit StringUtils.format("\"%sk\" == countryIsoCode", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), ExprMacroTable.nil() ).toFilter(); - - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1303,22 +1154,13 @@ public void test_makeCursors_factToRegionToCountryLeft() factToRegion(JoinType.LEFT), regionToCountry(JoinType.LEFT) ); - - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1383,20 +1225,13 @@ public void test_makeCursors_factToCountryAlwaysTrue() ); Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1450,21 +1285,13 @@ public void test_makeCursors_factToCountryAlwaysFalse() Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1501,21 +1328,13 @@ public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1569,21 +1388,13 @@ public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1629,21 +1440,13 @@ public void test_makeCursors_factToCountryUsingVirtualColumn() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - virtualColumns, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1695,21 +1498,13 @@ public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - virtualColumns, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1753,21 +1548,13 @@ public void test_makeCursors_factToCountryUsingExpression() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1810,21 +1597,13 @@ public void test_makeCursors_factToCountryUsingExpressionUsingLookup() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1869,22 +1648,13 @@ public void test_makeCursors_factToRegionTheWrongWay() ); Filter filter = new SelectorDimFilter("regionIsoCode", "VA", null).toFilter(); - - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1930,21 +1700,13 @@ public void test_makeCursors_errorOnNonEquiJoin() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1976,21 +1738,13 @@ public void test_makeCursors_errorOnNonEquiJoinUsingLookup() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -2022,21 +1776,13 @@ public void test_makeCursors_errorOnNonKeyBasedJoin() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -2067,21 +1813,13 @@ public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -2100,21 +1838,13 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRows() Filter originalFilter = new SelectorFilter("page", "this matches nothing"); List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( originalFilter, Intervals.ETERNITY, @@ -2139,21 +1869,13 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRowsUsingLoo { Filter originalFilter = new SelectorFilter("page", "this matches nothing"); List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( originalFilter, Intervals.ETERNITY, @@ -2177,34 +1899,21 @@ public void test_makeCursors_originalFilterDoesNotMatchPreAnalysis_shouldThrowIS { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); Filter filter = new SelectorFilter("page", "this matches nothing"); - try { - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - preAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - Assert.fail(); - } - catch (ISE e) { - Assert.assertTrue(e.getMessage().startsWith("Filter provided to cursor [")); - } + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysisGroup + ).makeCursors( + filter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ); } + } diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java index 3ff19e4d5918..02da30855eb9 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java @@ -24,10 +24,8 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; -import org.apache.druid.segment.join.filter.JoinableClauses; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.timeline.SegmentId; import org.hamcrest.CoreMatchers; @@ -44,6 +42,14 @@ public class HashJoinSegmentTest { + private JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, + false + ); + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -82,20 +88,14 @@ public void setUp() throws IOException ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + DEFAULT_JOIN_FILTER_REWRITE_CONFIG ); hashJoinSegment = new HashJoinSegment( baseSegment, joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); } @@ -107,20 +107,14 @@ public void test_constructor_noClauses() List joinableClauses = ImmutableList.of(); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + DEFAULT_JOIN_FILTER_REWRITE_CONFIG ); final HashJoinSegment ignored = new HashJoinSegment( baseSegment, joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); } diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java index 965dd2302f11..e77198464c3d 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java @@ -46,6 +46,8 @@ import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.segment.join.filter.JoinFilterSplit; import org.apache.druid.segment.join.filter.JoinableClauses; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -66,24 +68,13 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new SelectorFilter("channel", "#en.wikipedia"), - null, - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -116,6 +107,19 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() new Object[]{"History of Fourems", "Fourems Province", "Fourems"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new SelectorFilter("channel", "#en.wikipedia"), + null, + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -142,24 +146,13 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName regionExprToCountry ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new SelectorFilter("rtc.countryName", "United States"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -179,6 +172,19 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName new Object[]{"Cream Soda", "Ainigriv", "United States"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("rtc.countryName", "United States"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -196,29 +202,13 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("US"), null, null).toFilter() - ) - ), - new SelectorFilter("rtc.countryName", "United States"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -242,6 +232,24 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#en.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("US"), null, null).toFilter() + ) + ), + new SelectorFilter("rtc.countryName", "United States"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -261,33 +269,14 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("countryIsoCode", null), - new SelectorFilter("countryNumber", null), - new SelectorFilter("rtc.countryName", null), - new SelectorFilter("r1.regionName", null) - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -312,8 +301,29 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() ) : ImmutableList.of() // when not running in SQL compatible mode, countryNumber does not have nulls ); - } + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("countryIsoCode", null), + new SelectorFilter("countryNumber", null), + new SelectorFilter("rtc.countryName", null), + new SelectorFilter("r1.regionName", null) + ) + ), + ImmutableSet.of() + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } + @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns() { @@ -331,30 +341,14 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new SelectorFilter("baseTableInvalidColumn", "abcd"), - new AndFilter( - ImmutableList.of( - new SelectorFilter("baseTableInvalidColumn2", null), - new SelectorFilter("rtc.invalidColumn", "abcd"), - new SelectorFilter("r1.invalidColumn", "abcd") - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -372,6 +366,25 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new SelectorFilter("baseTableInvalidColumn", "abcd"), + new AndFilter( + ImmutableList.of( + new SelectorFilter("baseTableInvalidColumn2", null), + new SelectorFilter("rtc.invalidColumn", "abcd"), + new SelectorFilter("r1.invalidColumn", "abcd") + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -388,15 +401,12 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); VirtualColumns virtualColumns = VirtualColumns.create( @@ -410,14 +420,6 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC ) ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new SelectorFilter("v1", "virtual-column-#en.wikipedia"), - null, - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -449,6 +451,19 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC new Object[]{"History of Fourems", "Fourems Province", "Fourems"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new SelectorFilter("v1", "virtual-column-#en.wikipedia"), + null, + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + virtualColumns + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -472,29 +487,13 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu factToRegion(JoinType.LEFT) )); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - virtualColumns, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses.getJoinableClauses(), - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new SelectorFilter("v0", "VIRGINIA"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -513,8 +512,20 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu new Object[]{"Old Anatolian Turkish", "VIRGINIA"} ) ); - } + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("v0", "VIRGINIA"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses.getJoinableClauses(), + virtualColumns + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } @Test public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlreadyPushDownVariety() @@ -581,14 +592,31 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Les Argonautes", "Quebec", "Canada"} + ) ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -655,27 +683,13 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready ), ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"Les Argonautes", "Quebec", "Canada"} - ) - ); } @Test @@ -705,15 +719,32 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan new SelectorFilter("rtc.countryName", "States United") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Old Anatolian Turkish", "Ainigriv", "States United"} + ) ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -726,7 +757,12 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan new SelectorFilter("rtc.countryName", "States United"), ImmutableSet.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", @@ -745,25 +781,6 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() .iterator().next(); compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"Old Anatolian Turkish", "Ainigriv", "States United"} - ) - ); } @Test @@ -795,17 +812,15 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode"); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -846,14 +861,32 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"유희왕 GX", "Seoul", "Republic of Korea"}, + new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} + ) ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -896,28 +929,14 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() ), ImmutableSet.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"유희왕 GX", "Seoul", "Republic of Korea"}, - new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} - ) + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -946,15 +965,32 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - filter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + filter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"President of India", "Usca"}, + new Object[]{"Otjiwarongo Airport", "Usca"}, + new Object[]{"Carlo Curti", "Usca"} + ) ); ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( @@ -975,6 +1011,11 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel expectedVirtualColumn ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + filter, + joinableClauses, + VirtualColumns.EMPTY + ); JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals( expectedFilterSplit.getBaseTableFilter(), @@ -987,26 +1028,6 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() .iterator().next(); compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"President of India", "Usca"}, - new Object[]{"Otjiwarongo Airport", "Usca"}, - new Object[]{"Carlo Curti", "Usca"} - ) - ); } @Test @@ -1035,15 +1056,32 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - filter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + filter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" + ), + ImmutableList.of( + new Object[]{"President of India", "Usca"}, + new Object[]{"Otjiwarongo Airport", "Usca"}, + new Object[]{"Carlo Curti", "Usca"} + ) ); ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( @@ -1064,6 +1102,11 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel expectedVirtualColumn ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + filter, + joinableClauses, + VirtualColumns.EMPTY + ); JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals( expectedFilterSplit.getBaseTableFilter(), @@ -1076,26 +1119,6 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() .iterator().next(); compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" - ), - ImmutableList.of( - new Object[]{"President of India", "Usca"}, - new Object[]{"Otjiwarongo Airport", "Usca"}, - new Object[]{"Carlo Curti", "Usca"} - ) - ); } @Test @@ -1108,28 +1131,12 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#de.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("DE"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1152,6 +1159,24 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new Object[]{"Diskussion:Sebastian Schulz", "DE", 3L, "DE", "Germany", 3L} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#de.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("DE"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1164,28 +1189,13 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Germany") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#de.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("DE"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Germany"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1207,6 +1217,24 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new Object[]{"Diskussion:Sebastian Schulz", "DE", 3L, "DE", "Germany"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#de.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("DE"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Germany"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1219,28 +1247,13 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1261,6 +1274,24 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1273,28 +1304,13 @@ public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNo new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "NO MATCH") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - FalseFilter.instance(), - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "NO MATCH") - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1315,6 +1331,25 @@ public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNo ), ImmutableList.of() ); + + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + FalseFilter.instance(), + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "NO MATCH") + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1327,29 +1362,14 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) - ) - ), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1368,6 +1388,24 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1380,28 +1418,13 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new InDimFilter("countryNumber", ImmutableSet.of("0"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). @@ -1430,6 +1453,24 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new Object[]{"Peremptory norm", "AU", "AU", "Australia", 0L} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#en.wikipedia"), + new InDimFilter("countryNumber", ImmutableSet.of("0"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1442,28 +1483,13 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", "Australia") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new InDimFilter("countryNumber", ImmutableSet.of("0"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", "Australia"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). @@ -1491,6 +1517,24 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new Object[]{"Peremptory norm", "AU", "0", "Australia"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#en.wikipedia"), + new InDimFilter("countryNumber", ImmutableSet.of("0"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", "Australia"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1503,29 +1547,14 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null) - ) - ), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1544,6 +1573,24 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1556,29 +1603,14 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", null) - ) - ), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1596,6 +1628,24 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1608,28 +1658,13 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa ) ); List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - filter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#es.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("SV"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "El Salvador"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1652,6 +1687,24 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa new Object[]{"Wendigo", "SV", 12L, "SV", "El Salvador", 12L} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#es.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("SV"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "El Salvador"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + filter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1664,29 +1717,14 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa ) ); List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.FULL)); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - filter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#es.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("SV"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "El Salvador"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( filter, @@ -1707,6 +1745,24 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa new Object[]{"Wendigo", "SV", 12L, "SV", "El Salvador"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#es.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("SV"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "El Salvador"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + filter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1719,29 +1775,14 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) - ) - ), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1761,6 +1802,24 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1773,29 +1832,13 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup() new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) - ) - ), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1814,6 +1857,24 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup() ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1838,29 +1899,13 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR ); Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province"); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new InDimFilter("countryIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), - new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter() - ) - ), - new SelectorFilter("r1.regionName", "Fourems Province"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1879,6 +1924,25 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR new Object[]{"History of Fourems", "Fourems Province"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new InDimFilter("countryIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), + new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter() + ) + ), + new SelectorFilter("r1.regionName", "Fourems Province"), + ImmutableSet.of() + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1903,33 +1967,18 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR ); Filter originalFilter = new OrFilter( ImmutableList.of( - new SelectorFilter("r1.regionName", "Fourems Province"), - new SelectorFilter("r1.regionIsoCode", "AAAA") - ) - ); - - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new OrFilter( - ImmutableList.of( - new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), - new SelectorFilter("regionIsoCode", "AAAA") - ) - ), - originalFilter, - ImmutableSet.of() + new SelectorFilter("r1.regionName", "Fourems Province"), + new SelectorFilter("r1.regionIsoCode", "AAAA") + ) + ); + + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1948,6 +1997,25 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR new Object[]{"History of Fourems", "Fourems Province"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new OrFilter( + ImmutableList.of( + new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), + new SelectorFilter("regionIsoCode", "AAAA") + ) + ), + originalFilter, + ImmutableSet.of() + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @@ -1979,28 +2047,13 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new OrFilter( - ImmutableList.of( - new SelectorFilter("user", "Fourems Province"), - new SelectorFilter("regionIsoCode", "AAAA") - ) - ), - null, - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); // This query doesn't execute because regionName is not a key column, but we can still check the // filter rewrites. @@ -2024,8 +2077,25 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR ), ImmutableList.of() ); - } + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new OrFilter( + ImmutableList.of( + new SelectorFilter("user", "Fourems Province"), + new SelectorFilter("regionIsoCode", "AAAA") + ) + ), + null, + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePushDown() @@ -2036,28 +2106,21 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush )); Filter originalFilter = new SelectorFilter("page", "Peremptory norm"); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - false, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, + false + ) ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses.getJoinableClauses(), - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new SelectorFilter("page", "Peremptory norm"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -2077,6 +2140,19 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush new Object[]{"Peremptory norm", "New South Wales", "Australia"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("page", "Peremptory norm"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses.getJoinableClauses(), + VirtualColumns.EMPTY + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -2108,19 +2184,42 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe )) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - false, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + true, + false, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, + false + ) ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses.getJoinableClauses(), - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"President of India", "California", "United States"}, + new Object[]{"Otjiwarongo Airport", "California", "United States"}, + new Object[]{"DirecTV", "North Carolina", "United States"}, + new Object[]{"Carlo Curti", "California", "United States"}, + new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} + ) ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -2152,31 +2251,13 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe ), ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses.getJoinableClauses(), + VirtualColumns.EMPTY + ); JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"President of India", "California", "United States"}, - new Object[]{"Otjiwarongo Airport", "California", "United States"}, - new Object[]{"DirecTV", "North Carolina", "United States"}, - new Object[]{"Carlo Curti", "California", "United States"}, - new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} - ) - ); } @Test @@ -2284,15 +2365,11 @@ public boolean supportsRequiredColumnRewrite() regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); - + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); String rewrittenCountryIsoCodeColumnName = hasLhsExpressionInJoinCondition @@ -2324,6 +2401,25 @@ public boolean supportsRequiredColumnRewrite() expectedVirtualColumns = ImmutableSet.of(); } + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Didier Leclair", "Ontario", "Canada"} + ) + ); + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( ImmutableList.of( @@ -2401,27 +2497,13 @@ public boolean supportsRequiredColumnRewrite() ), expectedVirtualColumns ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"Didier Leclair", "Ontario", "Canada"} - ) - ); } @Test @@ -2439,14 +2521,30 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + ) ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -2469,8 +2567,60 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa originalFilter, ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } + + + @Test + public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryNameWithOldRewriteMode() + { + Filter originalFilter = new SelectorFilter("rtc.countryName", "United States"); + JoinableClause regionExprToCountry = new JoinableClause( + REGION_TO_COUNTRY_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format( + "reverse(\"%scountryIsoCode\") == \"%scountryIsoCode\"", + FACT_TO_REGION_PREFIX, + REGION_TO_COUNTRY_PREFIX + ), + REGION_TO_COUNTRY_PREFIX, + ExprMacroTable.nil() + ) + ); + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionExprToCountry + ); + + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, + true + ) + ); + + joinFilterPreAnalysisGroup.performAnalysisForOldRewriteMode( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysisGroup + ); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -2487,8 +2637,18 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa REGION_TO_COUNTRY_PREFIX + "countryName" ), ImmutableList.of( + new Object[]{"Cream Soda", "Ainigriv", "United States"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("rtc.countryName", "United States"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForOldRewriteMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -2499,20 +2659,4 @@ public void test_JoinFilterSplit_equals() .withNonnullFields("baseTableFilter", "pushDownVirtualColumns") .verify(); } - - private static JoinFilterPreAnalysis simplePreAnalysis( - List joinableClauses, - Filter originalFilter - ) - { - return JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); - } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java index 4fa521dbcbc9..cf33f33c9547 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java @@ -28,8 +28,8 @@ import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.query.planning.PreJoinableClause; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.junit.Assert; import org.junit.Rule; @@ -44,6 +44,14 @@ public class JoinablesTest { + private static final JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, + QueryContexts.DEFAULT_JOIN_FILTER_REWRITE_USE_OLD_REWRITE_MODE + ); + @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -99,12 +107,9 @@ public void test_createSegmentMapFn_noClauses() ImmutableList.of(), NoopJoinableFactory.INSTANCE, new AtomicLong(), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, null, - VirtualColumns.EMPTY + null ); Assert.assertSame(Function.identity(), segmentMapFn); @@ -128,12 +133,9 @@ public void test_createSegmentMapFn_unusableClause() ImmutableList.of(clause), NoopJoinableFactory.INSTANCE, new AtomicLong(), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, null, - VirtualColumns.EMPTY + null ); } @@ -165,12 +167,9 @@ public void test_createSegmentMapFn_usableClause() } }, new AtomicLong(), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, null, - VirtualColumns.EMPTY + null ); Assert.assertNotSame(Function.identity(), segmentMapFn); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 504daf3d60a0..ca96730b1748 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -60,6 +60,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.Joinables; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; import org.apache.druid.timeline.SegmentId; @@ -170,15 +171,20 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final throw new ISE("Cannot handle subquery: %s", analysis.getDataSource()); } + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( + QueryContexts.getEnableJoinFilterPushDown(query), + QueryContexts.getEnableJoinFilterRewrite(query), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), + QueryContexts.getJoinFilterRewriteMaxSize(query), + QueryContexts.getUseJoinFilterRewriteOldRewriteMode(query) + ); + // segmentMapFn maps each base Segment into a joined Segment if necessary. final Function segmentMapFn = Joinables.createSegmentMapFn( analysis.getPreJoinableClauses(), joinableFactory, cpuTimeAccumulator, - QueryContexts.getEnableJoinFilterPushDown(query), - QueryContexts.getEnableJoinFilterRewrite(query), - QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), - QueryContexts.getJoinFilterRewriteMaxSize(query), + joinFilterRewriteConfig, query.getFilter() == null ? null : query.getFilter().toFilter(), query.getVirtualColumns() ); diff --git a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java index d7f39adaa4ba..174daae75f23 100644 --- a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java @@ -39,6 +39,7 @@ import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.Joinables; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.joda.time.Interval; import java.util.HashSet; @@ -93,14 +94,18 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final final Query prioritizedAndLaned = prioritizeAndLaneQuery(query, segments); final AtomicLong cpuAccumulator = new AtomicLong(0L); - final Function segmentMapFn = Joinables.createSegmentMapFn( - analysis.getPreJoinableClauses(), - joinableFactory, - cpuAccumulator, + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( QueryContexts.getEnableJoinFilterPushDown(prioritizedAndLaned), QueryContexts.getEnableJoinFilterRewrite(prioritizedAndLaned), QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(prioritizedAndLaned), QueryContexts.getJoinFilterRewriteMaxSize(prioritizedAndLaned), + QueryContexts.getUseJoinFilterRewriteOldRewriteMode(prioritizedAndLaned) + ); + final Function segmentMapFn = Joinables.createSegmentMapFn( + analysis.getPreJoinableClauses(), + joinableFactory, + cpuAccumulator, + joinFilterRewriteConfig, prioritizedAndLaned.getFilter() == null ? null : prioritizedAndLaned.getFilter().toFilter(), prioritizedAndLaned.getVirtualColumns() ); diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index ca26e0dd4f75..3d9b61b1f57e 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -58,6 +58,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.Joinables; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SetAndVerifyContextQueryRunner; import org.apache.druid.server.initialization.ServerConfig; @@ -193,15 +194,20 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable(); } + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( + QueryContexts.getEnableJoinFilterPushDown(query), + QueryContexts.getEnableJoinFilterRewrite(query), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), + QueryContexts.getJoinFilterRewriteMaxSize(query), + QueryContexts.getUseJoinFilterRewriteOldRewriteMode(query) + ); + // segmentMapFn maps each base Segment into a joined Segment if necessary. final Function segmentMapFn = Joinables.createSegmentMapFn( analysis.getPreJoinableClauses(), joinableFactory, cpuTimeAccumulator, - QueryContexts.getEnableJoinFilterPushDown(query), - QueryContexts.getEnableJoinFilterRewrite(query), - QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), - QueryContexts.getJoinFilterRewriteMaxSize(query), + joinFilterRewriteConfig, query.getFilter() == null ? null : query.getFilter().toFilter(), query.getVirtualColumns() ); diff --git a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java index cc3a406cfe78..d0697abc29eb 100644 --- a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java @@ -47,6 +47,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.Joinables; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; @@ -139,14 +140,19 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final throw new ISE("Cannot handle subquery: %s", analysis.getDataSource()); } - final Function segmentMapFn = Joinables.createSegmentMapFn( - analysis.getPreJoinableClauses(), - joinableFactory, - new AtomicLong(), + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( QueryContexts.getEnableJoinFilterPushDown(query), QueryContexts.getEnableJoinFilterRewrite(query), QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), QueryContexts.getJoinFilterRewriteMaxSize(query), + QueryContexts.getUseJoinFilterRewriteOldRewriteMode(query) + ); + + final Function segmentMapFn = Joinables.createSegmentMapFn( + analysis.getPreJoinableClauses(), + joinableFactory, + new AtomicLong(), + joinFilterRewriteConfig, query.getFilter() == null ? null : query.getFilter().toFilter(), query.getVirtualColumns() ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 94edd222dc67..85665d09bf4f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -31,7 +31,6 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.StringUtils; @@ -11484,81 +11483,77 @@ public void testTimeExtractWithTooFewArguments() throws Exception @Test @Parameters(source = QueryContextForJoinProvider.class) - public void testNestedGroupByOnInlineDataSourceWithFilterIsNotSupported(Map queryContext) throws Exception + public void testNestedGroupByOnInlineDataSourceWithFilter(Map queryContext) throws Exception { - try { - testQuery( - "with abc as" - + "(" - + " SELECT dim1, m2 from druid.foo where \"__time\" >= '2001-01-02'" - + ")" - + ", def as" - + "(" - + " SELECT t1.dim1, SUM(t2.m2) as \"metricSum\" " - + " from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1" - + " where t1.dim1='def'" - + " group by 1" - + ")" - + "SELECT count(*) from def", - queryContext, - ImmutableList.of( - GroupByQuery - .builder() - .setDataSource( - GroupByQuery - .builder() - .setDataSource( - join( - new QueryDataSource( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Intervals.of("2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) - .columns("dim1", "m2") - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(queryContext) - .build() - ), - new QueryDataSource( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Intervals.of("2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) - .columns("dim1", "m2") - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(queryContext) - .build() - ), - "j0", - equalsCondition( - DruidExpression.fromColumn("dim1"), - DruidExpression.fromColumn("j0.dim1") - ), - JoinType.INNER - ) - ) - .setGranularity(Granularities.ALL) - .setInterval(querySegmentSpec(Filtration.eternity())) - .build() - ) - .setGranularity(Granularities.ALL) - .setInterval(querySegmentSpec(Filtration.eternity())) - .build() - ), - ImmutableList.of(new Object[] {1}) - ); - Assert.fail("Expected an ISE to be thrown"); - } - catch (RuntimeException e) { - Throwable cause = e.getCause(); - boolean foundISE = false; - while (cause != null) { - if (cause instanceof ISE) { - foundISE = true; - break; - } - cause = cause.getCause(); - } - Assert.assertTrue(foundISE); - } + // Cannot vectorize due to virtual columns. + cannotVectorize(); + + testQuery( + "with abc as" + + "(" + + " SELECT dim1, m2 from druid.foo where \"__time\" >= '2001-01-02'" + + ")" + + ", def as" + + "(" + + " SELECT t1.dim1, SUM(t2.m2) as \"metricSum\" " + + " from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1" + + " where t1.dim1='def'" + + " group by 1" + + ")" + + "SELECT count(*) from def", + queryContext, + ImmutableList.of( + GroupByQuery + .builder() + .setDataSource( + GroupByQuery + .builder() + .setDataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Intervals.of("2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) + .columns("dim1") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Intervals.of("2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) + .columns("dim1", "m2") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromColumn("dim1"), + DruidExpression.fromColumn("j0.dim1") + ), + JoinType.INNER + ) + ) + .setGranularity(Granularities.ALL) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimFilter(selector("dim1", "def", null)) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0") + ) + ) + .setVirtualColumns(expressionVirtualColumn("v0", "'def'", ValueType.STRING)) + .build() + ) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setGranularity(Granularities.ALL) + .setInterval(querySegmentSpec(Filtration.eternity())) + .build() + ), + ImmutableList.of(new Object[] {1L}) + ); } @Test