From 37e150c0755c0efba609601b1122cd6bc8a271fb Mon Sep 17 00:00:00 2001 From: Jonathan Wei Date: Thu, 18 Jun 2020 21:32:29 -0700 Subject: [PATCH] Fix join filter rewrites with nested queries (#10015) * Fix join filter rewrites with nested queries * Fix test, inspection, coverage * Remove clauses from group key * Fix import order Co-authored-by: Gian Merlino --- .../benchmark/JoinAndLookupBenchmark.java | 80 +- .../druid/query/groupby/GroupByQuery.java | 2 +- .../apache/druid/query/scan/ScanQuery.java | 2 +- .../query/timeseries/TimeseriesQuery.java | 2 +- .../apache/druid/query/topn/TopNQuery.java | 2 +- .../druid/segment/join/HashJoinSegment.java | 13 +- .../join/HashJoinSegmentStorageAdapter.java | 25 +- .../apache/druid/segment/join/Joinables.java | 123 +- .../join/filter/JoinFilterAnalyzer.java | 26 +- .../join/filter/JoinFilterPreAnalysis.java | 42 +- .../rewrite/JoinFilterPreAnalysisGroup.java | 149 ++ .../rewrite/JoinFilterRewriteConfig.java | 84 + ...BaseHashJoinSegmentStorageAdapterTest.java | 54 +- .../HashJoinSegmentStorageAdapterTest.java | 490 ++---- .../segment/join/HashJoinSegmentTest.java | 39 +- .../segment/join/JoinFilterAnalyzerTest.java | 1395 +++++++++-------- .../druid/segment/join/JoinablesTest.java | 129 +- .../appenderator/SinkQuerySegmentWalker.java | 16 +- .../druid/server/LocalQuerySegmentWalker.java | 15 +- .../server/coordination/ServerManager.java | 16 +- .../server/TestClusterQuerySegmentWalker.java | 15 +- .../druid/sql/calcite/CalciteQueryTest.java | 143 +- 22 files changed, 1623 insertions(+), 1239 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterPreAnalysisGroup.java create mode 100644 processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterRewriteConfig.java 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 567a34842c9..132a3215927 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -49,9 +49,8 @@ import org.apache.druid.segment.join.JoinConditionAnalysis; 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; @@ -141,19 +140,20 @@ public class JoinAndLookupBenchmark ) ) ); - JoinFilterPreAnalysis preAnalysisLookupStringKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClausesLookupStringKey), - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + JoinFilterPreAnalysisGroup preAnalysisGroupLookupStringKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0 + ), + true ); + hashJoinLookupStringKeySegment = new HashJoinSegment( ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment), joinableClausesLookupStringKey, - preAnalysisLookupStringKey + preAnalysisGroupLookupStringKey ); List joinableClausesLookupLongKey = ImmutableList.of( @@ -168,19 +168,20 @@ public class JoinAndLookupBenchmark ) ) ); - JoinFilterPreAnalysis preAnalysisLookupLongKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClausesLookupLongKey), - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + + JoinFilterPreAnalysisGroup preAnalysisGroupLookupLongKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0 + ), + true ); hashJoinLookupLongKeySegment = new HashJoinSegment( ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment), joinableClausesLookupLongKey, - preAnalysisLookupLongKey + preAnalysisGroupLookupLongKey ); List joinableClausesIndexedTableStringKey = ImmutableList.of( @@ -195,19 +196,20 @@ public class JoinAndLookupBenchmark ) ) ); - JoinFilterPreAnalysis preAnalysisIndexedTableStringKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClausesIndexedTableStringKey), - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + + JoinFilterPreAnalysisGroup preAnalysisGroupIndexedStringKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0 + ), + true ); hashJoinIndexedTableStringKeySegment = new HashJoinSegment( ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment), joinableClausesIndexedTableStringKey, - preAnalysisIndexedTableStringKey + preAnalysisGroupIndexedStringKey ); List joinableClausesIndexedTableLonggKey = ImmutableList.of( @@ -222,19 +224,19 @@ public class JoinAndLookupBenchmark ) ) ); - JoinFilterPreAnalysis preAnalysisIndexedTableLongKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClausesIndexedTableLonggKey), - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + JoinFilterPreAnalysisGroup preAnalysisGroupIndexedLongKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0 + ), + true ); hashJoinIndexedTableLongKeySegment = new HashJoinSegment( ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment), joinableClausesIndexedTableLonggKey, - preAnalysisIndexedTableLongKey + preAnalysisGroupIndexedLongKey ); final Map countryCodeToNameMap = JoinTestHelper.createCountryIsoCodeToNameLookup().getMap(); 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 7bd2a847c08..462644425dc 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 @@ public class GroupByQuery extends BaseQuery 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 459b8326d21..be5d24b71b3 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 @@ public class ScanQuery extends BaseQuery 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 47ab8a8c670..6603ee05a51 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 @@ -108,8 +108,8 @@ public class TimeseriesQuery extends BaseQuery> 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 3c301e584af..f5bdec1982a 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 class TopNQuery extends BaseQuery> return TOPN; } - @Override @JsonProperty + @Override public VirtualColumns getVirtualColumns() { return virtualColumns; 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 3ca240604da..500e030c8c7 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 @@ -25,7 +25,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SegmentReference; 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; @@ -44,23 +44,24 @@ public class HashJoinSegment implements SegmentReference { private final SegmentReference 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( SegmentReference 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()) { @@ -93,7 +94,7 @@ public class HashJoinSegment implements SegmentReference @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 65c25d498b5..8e2518e5cc7 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 @@ package org.apache.druid.segment.join; 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.data.ListIndexed; 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.ArrayList; 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,23 @@ 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 +209,16 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter @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.isSingleLevelMode()) { + jfpa = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + } else { + jfpa = joinFilterPreAnalysisGroup.getAnalysis( filter, - joinFilterPreAnalysis.getOriginalFilter() + virtualColumns ); } + final List preJoinVirtualColumns = new ArrayList<>(); final List postJoinVirtualColumns = new ArrayList<>(); @@ -225,7 +228,7 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter 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 c9a9d2fc247..6ebe970ae29 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 @@ -20,17 +20,20 @@ package org.apache.druid.segment.join; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.JoinDataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.planning.PreJoinableClause; import org.apache.druid.segment.SegmentReference; -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; +import java.util.ArrayList; import java.util.Comparator; import java.util.List; import java.util.concurrent.atomic.AtomicLong; @@ -71,33 +74,19 @@ public class Joinables /** * 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 query The query being processed */ 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, + final Query query ) { // compute column correlations here and RHS correlated values @@ -108,21 +97,85 @@ public class Joinables return Function.identity(); } else { final JoinableClauses joinableClauses = JoinableClauses.createClauses(clauses, joinableFactory); - JoinFilterPreAnalysis jfpa = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - virtualColumns, - originalFilter, - enableFilterPushDown, - enableFilterRewrite, - enableRewriteValueColumnFilters, - filterRewriteMaxSize + + List joinQueryLevels = new ArrayList<>(); + Joinables.gatherAllJoinQueryLevels(query, joinQueryLevels); + + final JoinFilterPreAnalysisGroup preAnalysisGroup = new JoinFilterPreAnalysisGroup( + joinFilterRewriteConfig, + joinQueryLevels.size() <= 1 // use single-level mode if there's one or fewer query levels with joins ); - return baseSegment -> new HashJoinSegment(baseSegment, joinableClauses.getJoinableClauses(), jfpa); + + for (Query joinQuery : joinQueryLevels) { + preAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent( + joinQuery.getFilter() == null ? null : joinQuery.getFilter().toFilter(), + joinableClauses.getJoinableClauses(), + joinQuery.getVirtualColumns() + ); + } + + return baseSegment -> new HashJoinSegment(baseSegment, joinableClauses.getJoinableClauses(), preAnalysisGroup); } } ); } + /** + * Walks a query and its subqueries, finding any queries that read from a JoinDatasource, + * and adding them to a list provided by the caller. + * + * @param currentLevelQuery The query to analyze + * @param allJoinQueryLevels A mutable list provided by the caller. + */ + public static void gatherAllJoinQueryLevels(Query currentLevelQuery, List allJoinQueryLevels) + { + DataSource currentDatasource = currentLevelQuery.getDataSource(); + if (currentDatasource instanceof QueryDataSource) { + gatherAllJoinQueryLevels( + ((QueryDataSource) currentDatasource).getQuery(), + allJoinQueryLevels + ); + } + if (currentDatasource instanceof JoinDataSource) { + allJoinQueryLevels.add(currentLevelQuery); + gatherAllJoinQueryLevelsJoinDatasourceHelper( + (JoinDataSource) currentDatasource, + allJoinQueryLevels + ); + } + } + + private static void gatherAllJoinQueryLevelsJoinDatasourceHelper( + JoinDataSource joinDatasource, + List allJoinQueryLevels + ) + { + if (joinDatasource.getLeft() instanceof QueryDataSource) { + gatherAllJoinQueryLevels( + ((QueryDataSource) joinDatasource.getLeft()).getQuery(), + allJoinQueryLevels + ); + } + if (joinDatasource.getLeft() instanceof JoinDataSource) { + gatherAllJoinQueryLevelsJoinDatasourceHelper( + (JoinDataSource) joinDatasource.getLeft(), + allJoinQueryLevels + ); + } + if (joinDatasource.getRight() instanceof QueryDataSource) { + gatherAllJoinQueryLevels( + ((QueryDataSource) joinDatasource.getRight()).getQuery(), + allJoinQueryLevels + ); + } + if (joinDatasource.getRight() instanceof JoinDataSource) { + gatherAllJoinQueryLevelsJoinDatasourceHelper( + (JoinDataSource) joinDatasource.getRight(), + allJoinQueryLevels + ); + } + } + /** * Check if any prefixes in the provided list duplicate or shadow each other. * 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 f4f7d31be1c..06475ad89e5 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.FalseFilter; 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 class JoinFilterAnalyzer 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 class JoinFilterAnalyzer preAnalysisBuilder .withNormalizedBaseTableClauses(normalizedBaseTableClauses) .withNormalizedJoinTableClauses(normalizedJoinTableClauses); - if (!enableFilterRewrite) { + if (!joinFilterRewriteConfig.isEnableFilterRewrite()) { return preAnalysisBuilder.build(); } @@ -146,8 +136,8 @@ public class JoinFilterAnalyzer 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 ae3731db77a..de842c3dde0 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.query.filter.Filter; 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 @@ import java.util.Set; * 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 @@ public class 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 @@ public class 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 class JoinFilterPreAnalysis public boolean isEnableFilterPushDown() { - return enableFilterPushDown; + return rewriteConfig.isEnableFilterPushDown(); } public boolean isEnableFilterRewrite() { - return enableFilterRewrite; + return rewriteConfig.isEnableFilterRewrite(); } public Equiconditions getEquiconditions() @@ -134,22 +132,23 @@ public class JoinFilterPreAnalysis */ 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 class JoinFilterPreAnalysis 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 class JoinFilterPreAnalysis 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 00000000000..01d4b1d21f5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterPreAnalysisGroup.java @@ -0,0 +1,149 @@ +/* + * 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.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * 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.) + * + * If there is only a single Filter, then this class does not use the analyses map, instead of using a single reference + * for efficiency reasons. + */ +public class JoinFilterPreAnalysisGroup +{ + private final JoinFilterRewriteConfig joinFilterRewriteConfig; + private final Map analyses; + private final boolean isSingleLevelMode; + + /** + * Hashing and comparing filters can be expensive for large filters, so if we're only dealing with + * a single level of join query, then we can be more efficient by using a single reference instead of a map. + */ + private JoinFilterPreAnalysis preAnalysisForSingleLevelMode; + + public JoinFilterPreAnalysisGroup( + JoinFilterRewriteConfig joinFilterRewriteConfig, + boolean isSingleLevelMode + ) + { + this.joinFilterRewriteConfig = joinFilterRewriteConfig; + this.analyses = new HashMap<>(); + this.isSingleLevelMode = isSingleLevelMode; + } + + public boolean isSingleLevelMode() + { + return isSingleLevelMode; + } + + public JoinFilterPreAnalysis computeJoinFilterPreAnalysisIfAbsent( + Filter filter, + List clauses, + VirtualColumns virtualColumns + ) + { + if (isSingleLevelMode) { + preAnalysisForSingleLevelMode = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinableClauses.fromList(clauses), + virtualColumns, + filter, + joinFilterRewriteConfig + ); + return preAnalysisForSingleLevelMode; + } + + JoinFilterPreAnalysisGroupKey key = new JoinFilterPreAnalysisGroupKey(filter, virtualColumns); + return analyses.computeIfAbsent( + key, + (groupKey) -> { + return JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinableClauses.fromList(clauses), + virtualColumns, + filter, + joinFilterRewriteConfig + ); + } + ); + } + + public JoinFilterPreAnalysis getAnalysis( + Filter filter, + VirtualColumns virtualColumns + ) + { + JoinFilterPreAnalysisGroupKey key = new JoinFilterPreAnalysisGroupKey(filter, virtualColumns); + return analyses.get(key); + } + + public JoinFilterPreAnalysis getPreAnalysisForSingleLevelMode() + { + return preAnalysisForSingleLevelMode; + } + + public static class JoinFilterPreAnalysisGroupKey + { + private final Filter filter; + private final VirtualColumns virtualColumns; + + public JoinFilterPreAnalysisGroupKey( + Filter filter, + VirtualColumns virtualColumns + ) + { + this.filter = filter; + 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(virtualColumns, that.virtualColumns); + } + + @Override + public int hashCode() + { + return Objects.hash(filter, 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 00000000000..ed948bcc14f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterRewriteConfig.java @@ -0,0 +1,84 @@ +/* + * 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; + + public JoinFilterRewriteConfig( + boolean enableFilterPushDown, + boolean enableFilterRewrite, + boolean enableRewriteValueColumnFilters, + long filterRewriteMaxSize + ) + { + this.enableFilterPushDown = enableFilterPushDown; + this.enableFilterRewrite = enableFilterRewrite; + this.enableRewriteValueColumnFilters = enableRewriteValueColumnFilters; + this.filterRewriteMaxSize = filterRewriteMaxSize; + } + + public boolean isEnableFilterPushDown() + { + return enableFilterPushDown; + } + + public boolean isEnableFilterRewrite() + { + return enableFilterRewrite; + } + + public boolean isEnableRewriteValueColumnFilters() + { + return enableRewriteValueColumnFilters; + } + + public long getFilterRewriteMaxSize() + { + return filterRewriteMaxSize; + } + +} 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 45befeba36e..5fd495340b8 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 @@ -24,12 +24,12 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.filter.Filter; 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; @@ -44,9 +44,17 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import java.io.IOException; +import java.util.List; public class BaseHashJoinSegmentStorageAdapterTest { + public static JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + ); + 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 +195,12 @@ public class BaseHashJoinSegmentStorageAdapterTest 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 +222,32 @@ public class BaseHashJoinSegmentStorageAdapterTest actualVirtualColumn.getParsedExpression().get().toString() ); } + + protected static JoinFilterPreAnalysisGroup makeDefaultConfigPreAnalysisGroup() + { + return new JoinFilterPreAnalysisGroup( + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + true + ); + } + + protected static JoinFilterPreAnalysisGroup makeDefaultConfigPreAnalysisGroup( + Filter originalFilter, + List joinableClauses, + VirtualColumns virtualColumns + ) + { + JoinFilterPreAnalysisGroup group = new JoinFilterPreAnalysisGroup( + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + true + ); + + group.computeJoinFilterPreAnalysisIfAbsent( + originalFilter, + joinableClauses, + virtualColumns + ); + + return group; + } } 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 2e863b2487a..ed6e1782f3a 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.ImmutableList; 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.VirtualColumns; 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,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -371,21 +362,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -438,21 +424,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag public void test_makeCursors_factToCountryInner() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.INNER)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -500,21 +481,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag public void test_makeCursors_factToCountryInnerUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.INNER)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -564,21 +540,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag // 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, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -634,21 +605,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag // 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, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -700,21 +666,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { 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, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -741,21 +702,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { 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, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -781,21 +737,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -824,21 +775,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.RIGHT)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -866,21 +812,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -909,21 +850,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.FULL)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -956,21 +892,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag null ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1003,21 +934,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag null ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1050,21 +976,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber", "10", null) ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1097,21 +1018,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Norway", null) ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1157,21 +1073,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ExprMacroTable.nil() ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1238,22 +1149,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag StringUtils.format("\"%sk\" == countryIsoCode", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), ExprMacroTable.nil() ).toFilter(); - - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1303,22 +1208,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag factToRegion(JoinType.LEFT), regionToCountry(JoinType.LEFT) ); - - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1383,20 +1282,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1450,21 +1345,17 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1501,21 +1392,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1569,21 +1455,17 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1629,21 +1511,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - virtualColumns, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + virtualColumns ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1695,21 +1572,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - virtualColumns, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + virtualColumns ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1753,21 +1625,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1810,21 +1677,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1869,22 +1731,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); Filter filter = new SelectorDimFilter("regionIsoCode", "VA", null).toFilter(); - - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1930,21 +1786,17 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1976,21 +1828,17 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -2022,21 +1870,17 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -2067,21 +1911,17 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -2100,21 +1940,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag Filter originalFilter = new SelectorFilter("page", "this matches nothing"); List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( originalFilter, Intervals.ETERNITY, @@ -2139,21 +1974,16 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { Filter originalFilter = new SelectorFilter("page", "this matches nothing"); List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( originalFilter, Intervals.ETERNITY, @@ -2176,35 +2006,25 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag public void test_makeCursors_originalFilterDoesNotMatchPreAnalysis_shouldThrowISE() { 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 - ); Filter filter = new SelectorFilter("page", "this matches nothing"); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, + joinableClauses, + VirtualColumns.EMPTY + ); - 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 966d62023ff..d04ebc83354 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 @@ -28,10 +28,8 @@ import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.StorageAdapter; -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.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; @@ -52,6 +50,13 @@ import java.util.Optional; public class HashJoinSegmentTest extends InitializedNullHandlingTest { + private JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + ); + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -129,14 +134,9 @@ public class HashJoinSegmentTest extends InitializedNullHandlingTest ) ); - 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, + true ); referencedSegment = ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment); @@ -188,7 +188,7 @@ public class HashJoinSegmentTest extends InitializedNullHandlingTest hashJoinSegment = new HashJoinSegment( testWrapper, joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ) { @Override @@ -213,20 +213,15 @@ public class HashJoinSegmentTest extends InitializedNullHandlingTest 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, + true ); final HashJoinSegment ignored = new HashJoinSegment( ReferenceCountingSegment.wrapRootGenerationSegment(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 965dd2302f1..893e0542b8c 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.JoinFilterAnalyzer; 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,25 +68,18 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new SelectorFilter("channel", "#en.wikipedia"), - null, - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -116,6 +111,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new Object[]{"History of Fourems", "Fourems Province", "Fourems"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new SelectorFilter("channel", "#en.wikipedia"), + null, + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -142,25 +146,18 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes regionExprToCountry ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new SelectorFilter("rtc.countryName", "United States"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -179,6 +176,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new Object[]{"Cream Soda", "Ainigriv", "United States"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("rtc.countryName", "United States"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -196,30 +202,19 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - 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() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -242,6 +237,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -261,33 +271,18 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - 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() - ); - - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -312,8 +307,26 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) : 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } + @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns() { @@ -331,31 +344,18 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - 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() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -372,6 +372,22 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -388,15 +404,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); VirtualColumns virtualColumns = VirtualColumns.create( @@ -410,14 +426,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - 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 +457,16 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -472,30 +490,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes factToRegion(JoinType.LEFT) )); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - virtualColumns, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses.getJoinableClauses(), + virtualColumns ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses.getJoinableClauses(), - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new SelectorFilter("v0", "VIRGINIA"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -513,8 +518,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new Object[]{"Old Anatolian Turkish", "VIRGINIA"} ) ); - } + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("v0", "VIRGINIA"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } @Test public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlreadyPushDownVariety() @@ -581,14 +595,34 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); 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 +689,10 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + 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,47 +722,18 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter("rtc.countryName", "States United") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( - joinableClauses, - originalFilter - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new InDimFilter("JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", ImmutableSet.of("SU"), null, null).toFilter() - ) - ), - new SelectorFilter("rtc.countryName", "States United"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - - ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( - "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", - "reverse(countryIsoCode)", - ValueType.STRING, - ExprMacroTable.nil() - ); - Assert.assertEquals( - expectedFilterSplit.getBaseTableFilter(), - actualFilterSplit.getBaseTableFilter() - ); - Assert.assertEquals( - expectedFilterSplit.getJoinTableFilter(), - actualFilterSplit.getJoinTableFilter() - ); - ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() - .iterator().next(); - compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -764,6 +752,38 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new Object[]{"Old Anatolian Turkish", "Ainigriv", "States United"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#en.wikipedia"), + new InDimFilter("JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", ImmutableSet.of("SU"), null, null).toFilter() + ) + ), + new SelectorFilter("rtc.countryName", "States United"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + + ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( + "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", + "reverse(countryIsoCode)", + ValueType.STRING, + ExprMacroTable.nil() + ); + Assert.assertEquals( + expectedFilterSplit.getBaseTableFilter(), + actualFilterSplit.getBaseTableFilter() + ); + Assert.assertEquals( + expectedFilterSplit.getJoinTableFilter(), + actualFilterSplit.getJoinTableFilter() + ); + ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() + .iterator().next(); + compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); } @Test @@ -795,9 +815,10 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode"); @@ -805,7 +826,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -846,14 +867,35 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); 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 +938,11 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), ImmutableSet.of() ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + 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"} - ) - ); } @Test @@ -946,48 +971,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, joinableClauses, - filter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( - "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", - "concat(countryIsoCode, regionIsoCode)", - ValueType.STRING, - ExprMacroTable.nil() - ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new InDimFilter("JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", ImmutableSet.of("USCA"), null, null).toFilter() - ) - ), - new SelectorFilter("c1.countryName", "Usca"), - ImmutableSet.of( - expectedVirtualColumn - ) - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals( - expectedFilterSplit.getBaseTableFilter(), - actualFilterSplit.getBaseTableFilter() - ); - Assert.assertEquals( - expectedFilterSplit.getJoinTableFilter(), - actualFilterSplit.getJoinTableFilter() - ); - ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() - .iterator().next(); - compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); - JoinTestHelper.verifyCursors( adapter.makeCursors( filter, @@ -1007,6 +1001,39 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new Object[]{"Carlo Curti", "Usca"} ) ); + + ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( + "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", + "concat(countryIsoCode, regionIsoCode)", + ValueType.STRING, + ExprMacroTable.nil() + ); + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#en.wikipedia"), + new InDimFilter("JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", ImmutableSet.of("USCA"), null, null).toFilter() + ) + ), + new SelectorFilter("c1.countryName", "Usca"), + ImmutableSet.of( + expectedVirtualColumn + ) + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals( + expectedFilterSplit.getBaseTableFilter(), + actualFilterSplit.getBaseTableFilter() + ); + Assert.assertEquals( + expectedFilterSplit.getJoinTableFilter(), + actualFilterSplit.getJoinTableFilter() + ); + ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() + .iterator().next(); + compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); } @Test @@ -1035,48 +1062,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, joinableClauses, - filter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( - "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", - "concat(countryIsoCode, regionIsoCode)", - ValueType.STRING, - ExprMacroTable.nil() - ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new InDimFilter("JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", ImmutableSet.of("USCA"), null, null).toFilter() - ) - ), - new SelectorFilter("c1.v", "Usca"), - ImmutableSet.of( - expectedVirtualColumn - ) - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals( - expectedFilterSplit.getBaseTableFilter(), - actualFilterSplit.getBaseTableFilter() - ); - Assert.assertEquals( - expectedFilterSplit.getJoinTableFilter(), - actualFilterSplit.getJoinTableFilter() - ); - ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() - .iterator().next(); - compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); - JoinTestHelper.verifyCursors( adapter.makeCursors( filter, @@ -1096,6 +1092,39 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new Object[]{"Carlo Curti", "Usca"} ) ); + + ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( + "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", + "concat(countryIsoCode, regionIsoCode)", + ValueType.STRING, + ExprMacroTable.nil() + ); + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#en.wikipedia"), + new InDimFilter("JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", ImmutableSet.of("USCA"), null, null).toFilter() + ) + ), + new SelectorFilter("c1.v", "Usca"), + ImmutableSet.of( + expectedVirtualColumn + ) + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals( + expectedFilterSplit.getBaseTableFilter(), + actualFilterSplit.getBaseTableFilter() + ); + Assert.assertEquals( + expectedFilterSplit.getJoinTableFilter(), + actualFilterSplit.getJoinTableFilter() + ); + ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() + .iterator().next(); + compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); } @Test @@ -1108,29 +1137,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - 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() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1152,6 +1169,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1164,29 +1196,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Germany") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - 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() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1207,6 +1227,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1219,14 +1254,35 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); 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", + "countryIsoCode", + "countryNumber", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of() ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -1239,28 +1295,10 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + 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", - "countryIsoCode", - "countryNumber", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" - ), - ImmutableList.of() - ); } @Test @@ -1273,29 +1311,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "NO MATCH") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - 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() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1315,6 +1341,22 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1327,29 +1369,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); 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 +1398,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1380,29 +1425,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - 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() - ); - 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). JoinTestHelper.verifyCursors( @@ -1430,6 +1463,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1442,29 +1490,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", "Australia") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - 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() - ); - 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). JoinTestHelper.verifyCursors( @@ -1491,6 +1527,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1503,29 +1554,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); 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 +1583,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1556,29 +1610,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); 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 +1638,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1608,29 +1665,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, joinableClauses, - filter + VirtualColumns.EMPTY ); 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 + "countryName", "El Salvador"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( filter, @@ -1652,6 +1697,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1664,29 +1724,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.FULL)); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, joinableClauses, - filter + VirtualColumns.EMPTY ); 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 +1755,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1719,29 +1782,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); 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 +1812,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1773,29 +1839,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); 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 +1868,21 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1838,30 +1907,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ); Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province"); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - 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() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1879,6 +1935,22 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1908,29 +1980,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new OrFilter( - ImmutableList.of( - new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), - new SelectorFilter("regionIsoCode", "AAAA") - ) - ), - originalFilter, - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1948,6 +2008,22 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @@ -1979,29 +2055,17 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new OrFilter( - ImmutableList.of( - new SelectorFilter("user", "Fourems Province"), - new SelectorFilter("regionIsoCode", "AAAA") - ) - ), - null, - ImmutableSet.of() - ); - 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. expectedException.expect(IAE.class); @@ -2024,8 +2088,22 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), 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.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePushDown() @@ -2036,29 +2114,27 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes )); 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 + ), + true ); + joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent( + originalFilter, + joinableClauses.getJoinableClauses(), + VirtualColumns.EMPTY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses.getJoinableClauses(), - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new SelectorFilter("page", "Peremptory norm"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -2077,6 +2153,16 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes new Object[]{"Peremptory norm", "New South Wales", "Australia"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("page", "Peremptory norm"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -2108,19 +2194,48 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes )) ) ); - 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 + ), + true ); + joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent( + originalFilter, + joinableClauses.getJoinableClauses(), + VirtualColumns.EMPTY + ); + 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 +2267,10 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + 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 +2378,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); String rewrittenCountryIsoCodeColumnName = hasLhsExpressionInJoinCondition @@ -2324,6 +2418,25 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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 +2514,10 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ), expectedVirtualColumns ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + 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 +2535,34 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); + 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 +2585,57 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes originalFilter, ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } + + + @Test + public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryNameWithMultiLevelMode() + { + 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 + ), + false + ); + + joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysisGroup + ); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -2487,8 +2652,22 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes 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.getAnalysis( + originalFilter, + VirtualColumns.EMPTY + ); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -2500,19 +2679,13 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes .verify(); } - private static JoinFilterPreAnalysis simplePreAnalysis( - List joinableClauses, - Filter originalFilter - ) + + @Test + public void test_JoinFilterPreAnalysisGroupKey_equals() { - return JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - JoinableClauses.fromList(joinableClauses), - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + EqualsVerifier.forClass(JoinFilterPreAnalysisGroup.JoinFilterPreAnalysisGroupKey.class) + .usingGetClass() + .withNonnullFields("filter", "virtualColumns") + .verify(); } } 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 f7c4f4b415b..c418f7fd45e 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 @@ -22,22 +22,34 @@ package org.apache.druid.segment.join; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.DataSource; +import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.LookupDataSource; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.TestQuery; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.MapLookupExtractor; +import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.planning.PreJoinableClause; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.SegmentReference; -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; import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; @@ -45,6 +57,13 @@ import java.util.function.Function; 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 + ); + @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -100,12 +119,8 @@ public class JoinablesTest 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, - null, - VirtualColumns.EMPTY + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + null ); Assert.assertSame(Function.identity(), segmentMapFn); @@ -129,12 +144,8 @@ public class JoinablesTest 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, - null, - VirtualColumns.EMPTY + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + null ); } @@ -177,12 +188,13 @@ public class JoinablesTest } }, 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, - null, - VirtualColumns.EMPTY + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + new TestQuery( + new TableDataSource("test"), + new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), + false, + new HashMap() + ) ); Assert.assertNotSame(Function.identity(), segmentMapFn); @@ -236,4 +248,83 @@ public class JoinablesTest Joinables.checkPrefixesForDuplicatesAndShadowing(prefixes); } + + @Test + public void test_gatherAllJoinQueryLevels() + { + Query query1 = new GroupByQuery.Builder() + .addDimension("dim1") + .setDataSource( + JoinDataSource.create( + new NoopDataSource(), + new NoopDataSource(), + "111", + "1", + JoinType.LEFT, + TestExprMacroTable.INSTANCE + ) + ) + .setInterval("1999/2000") + .setGranularity(Granularities.YEAR) + .build(); + + Query query2 = new GroupByQuery.Builder() + .addDimension("dim2") + .setDataSource( + JoinDataSource.create( + new NoopDataSource(), + new NoopDataSource(), + "222", + "1", + JoinType.LEFT, + TestExprMacroTable.INSTANCE + ) + ) + .setInterval("1999/2000") + .setGranularity(Granularities.YEAR) + .build(); + + QueryDataSource queryDataSource1 = new QueryDataSource(query1); + QueryDataSource queryDataSource2 = new QueryDataSource(query2); + + Query query3 = new GroupByQuery.Builder() + .addDimension("dim2") + .setDataSource( + JoinDataSource.create( + JoinDataSource.create( + queryDataSource1, + new NoopDataSource(), + "444", + "4", + JoinType.LEFT, + TestExprMacroTable.INSTANCE + ), + queryDataSource2, + "333", + "3", + JoinType.LEFT, + TestExprMacroTable.INSTANCE + ) + ) + .setInterval("1999/2000") + .setGranularity(Granularities.YEAR) + .build(); + + Query queryOuter = new GroupByQuery.Builder() + .addDimension("dim") + .setDataSource( + new QueryDataSource(query3) + ) + .setInterval("1999/2000") + .setGranularity(Granularities.YEAR) + .build(); + + List joinQueryLevels = new ArrayList<>(); + Joinables.gatherAllJoinQueryLevels(queryOuter, joinQueryLevels); + + Assert.assertEquals( + ImmutableList.of(query3, query1, query2), + joinQueryLevels + ); + } } 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 4527287b4f6..b46bd743180 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 @@ -59,6 +59,7 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.SegmentReference; 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; @@ -169,17 +170,20 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker 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) + ); + // 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), - query.getFilter() == null ? null : query.getFilter().toFilter(), - query.getVirtualColumns() + joinFilterRewriteConfig, + query ); Iterable> perSegmentRunners = Iterables.transform( 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 8283524040f..dbd043f52bf 100644 --- a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.SegmentReference; 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; @@ -100,16 +101,18 @@ public class LocalQuerySegmentWalker implements QuerySegmentWalker final Query prioritizedAndLaned = prioritizeAndLaneQuery(query, segments); final AtomicLong cpuAccumulator = new AtomicLong(0L); + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( + QueryContexts.getEnableJoinFilterPushDown(prioritizedAndLaned), + QueryContexts.getEnableJoinFilterRewrite(prioritizedAndLaned), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(prioritizedAndLaned), + QueryContexts.getJoinFilterRewriteMaxSize(prioritizedAndLaned) + ); final Function segmentMapFn = Joinables.createSegmentMapFn( analysis.getPreJoinableClauses(), joinableFactory, cpuAccumulator, - QueryContexts.getEnableJoinFilterPushDown(prioritizedAndLaned), - QueryContexts.getEnableJoinFilterRewrite(prioritizedAndLaned), - QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(prioritizedAndLaned), - QueryContexts.getJoinFilterRewriteMaxSize(prioritizedAndLaned), - prioritizedAndLaned.getFilter() == null ? null : prioritizedAndLaned.getFilter().toFilter(), - prioritizedAndLaned.getVirtualColumns() + joinFilterRewriteConfig, + query ); final QueryRunnerFactory> queryRunnerFactory = conglomerate.findFactory(prioritizedAndLaned); 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 086673fabc1..b2bef330262 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 @@ -57,6 +57,7 @@ import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; 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; @@ -192,17 +193,20 @@ public class ServerManager implements QuerySegmentWalker return new NoopQueryRunner<>(); } + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( + QueryContexts.getEnableJoinFilterPushDown(query), + QueryContexts.getEnableJoinFilterRewrite(query), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), + QueryContexts.getJoinFilterRewriteMaxSize(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), - query.getFilter() == null ? null : query.getFilter().toFilter(), - query.getVirtualColumns() + joinFilterRewriteConfig, + query ); FunctionalIterable> queryRunners = FunctionalIterable 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 e41ad268d16..c11bb817916 100644 --- a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java @@ -48,6 +48,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; 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,17 +140,19 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker && !toolChest.canPerformSubquery(((QueryDataSource) analysis.getDataSource()).getQuery())) { 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) + ); final Function segmentMapFn = Joinables.createSegmentMapFn( analysis.getPreJoinableClauses(), joinableFactory, new AtomicLong(), - QueryContexts.getEnableJoinFilterPushDown(query), - QueryContexts.getEnableJoinFilterRewrite(query), - QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), - QueryContexts.getJoinFilterRewriteMaxSize(query), - query.getFilter() == null ? null : query.getFilter().toFilter(), - query.getVirtualColumns() + joinFilterRewriteConfig, + query ); final QueryRunner baseRunner = new FinalizeResultsQueryRunner<>( 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 62c1519473f..1606ef7e7b9 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.annotations.UsedByJUnitParamsRunner; 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; @@ -11917,79 +11916,75 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Parameters(source = QueryContextForJoinProvider.class) 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