From ddae773e3cd06d4d65fd0d4e008331e8d019be6a Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 9 Mar 2020 19:37:53 -0700 Subject: [PATCH 1/8] More efficient join filter rewrites --- .../java/org/apache/druid/query/Query.java | 5 + .../org/apache/druid/query/QueryContexts.java | 24 +- .../druid/query/groupby/GroupByQuery.java | 1 + .../apache/druid/query/scan/ScanQuery.java | 1 + .../query/timeseries/TimeseriesQuery.java | 1 + .../apache/druid/query/topn/TopNQuery.java | 1 + .../druid/segment/join/HashJoinSegment.java | 12 +- .../join/HashJoinSegmentStorageAdapter.java | 39 +- .../apache/druid/segment/join/Joinable.java | 4 +- .../apache/druid/segment/join/Joinables.java | 22 +- .../join/filter/JoinFilterAnalyzer.java | 773 +++++++++------- .../JoinFilterColumnCorrelationAnalysis.java | 11 + .../join/filter/JoinFilterPreAnalysis.java | 113 +++ .../segment/join/lookup/LookupJoinable.java | 9 +- .../join/table/IndexedTableJoinable.java | 15 +- ...BaseHashJoinSegmentStorageAdapterTest.java | 17 +- .../HashJoinSegmentStorageAdapterTest.java | 677 ++++++++++---- .../segment/join/HashJoinSegmentTest.java | 64 +- .../segment/join/JoinFilterAnalyzerTest.java | 839 ++++++++++++------ .../druid/segment/join/JoinablesTest.java | 19 +- .../appenderator/SinkQuerySegmentWalker.java | 6 +- .../server/coordination/ServerManager.java | 6 +- .../SpecificSegmentsQuerySegmentWalker.java | 6 +- 23 files changed, 1830 insertions(+), 835 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index 31e21e9d4c6e..702105b23129 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -37,6 +37,7 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.VirtualColumns; import org.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Interval; @@ -158,4 +159,8 @@ default Query withLane(String lane) return withOverriddenContext(ImmutableMap.of(QueryContexts.LANE_KEY, lane)); } + default VirtualColumns getVirtualColumns() + { + return VirtualColumns.EMPTY; + } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index ba8e12b09eae..b2b88023b609 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -47,7 +47,9 @@ public class QueryContexts public static final String VECTORIZE_KEY = "vectorize"; public static final String VECTOR_SIZE_KEY = "vectorSize"; public static final String JOIN_FILTER_PUSH_DOWN_KEY = "enableJoinFilterPushDown"; - public static final String JOIN_FILTER_REWRITE_KEY = "enableJoinFilterRewrite"; + public static final String JOIN_FILTER_REWRITE_ENABLE_KEY = "enableJoinFilterRewrite"; + public static final String JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY = "enableJoinFilterRewriteValueColumnFilters"; + public static final String JOIN_FILTER_REWRITE_MAX_SIZE_KEY = "joinFilterRewriteMaxSize"; public static final boolean DEFAULT_BY_SEGMENT = false; public static final boolean DEFAULT_POPULATE_CACHE = true; @@ -61,7 +63,9 @@ public class QueryContexts public static final long NO_TIMEOUT = 0; public static final boolean DEFAULT_ENABLE_PARALLEL_MERGE = true; public static final boolean DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN = true; - public static final boolean DEFAULT_ENABLE_JOIN_FILTER_REWRITE = false; + public static final boolean DEFAULT_ENABLE_JOIN_FILTER_REWRITE = true; + public static final boolean DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS = false; + public static final long DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY = 10000; @SuppressWarnings("unused") // Used by Jackson serialization public enum Vectorize @@ -227,6 +231,19 @@ public static int getParallelMergeParallelism(Query query, int defaultVal { return parseInt(query, BROKER_PARALLELISM, defaultValue); } + public static boolean getEnableJoinFilterRewriteValueColumnFilters(Query query) + { + return parseBoolean( + query, + JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, + DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS + ); + } + + public static long getJoinFilterRewriteMaxSize(Query query) + { + return parseLong(query, JOIN_FILTER_REWRITE_MAX_SIZE_KEY, DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY); + } public static boolean getEnableJoinFilterPushDown(Query query) { @@ -235,9 +252,10 @@ public static boolean getEnableJoinFilterPushDown(Query query) public static boolean getEnableJoinFilterRewrite(Query query) { - return parseBoolean(query, JOIN_FILTER_REWRITE_KEY, DEFAULT_ENABLE_JOIN_FILTER_REWRITE); + return parseBoolean(query, JOIN_FILTER_REWRITE_ENABLE_KEY, DEFAULT_ENABLE_JOIN_FILTER_REWRITE); } + public static Query withMaxScatterGatherBytes(Query query, long maxScatterGatherBytesLimit) { Object obj = query.getContextValue(MAX_SCATTER_GATHER_BYTES_KEY); 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 4c17f16b03af..7bd2a847c081 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,6 +249,7 @@ private List> verifySubtotalsSpec( return subtotalsSpec; } + @Override @JsonProperty public VirtualColumns getVirtualColumns() { 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 719f5f27e6f7..459b8326d217 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,6 +183,7 @@ private Integer validateAndGetMaxSegmentPartitionsOrderedInMemory() return maxSegmentPartitionsOrderedInMemory; } + @Override @JsonProperty public VirtualColumns getVirtualColumns() { 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 74d2a9306768..9cbcc7c6ef17 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -103,6 +103,7 @@ public String getType() return Query.TIMESERIES; } + @Override @JsonProperty public VirtualColumns getVirtualColumns() { 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 c77011c0325d..3c301e584af8 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,6 +113,7 @@ public String getType() return TOPN; } + @Override @JsonProperty public VirtualColumns getVirtualColumns() { 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 6ad0aa9f38ee..854cd1f86c1e 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java @@ -24,6 +24,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -40,8 +41,7 @@ public class HashJoinSegment extends AbstractSegment { private final Segment baseSegment; private final List clauses; - private final boolean enableFilterPushDown; - private final boolean enableFilterRewrite; + private final JoinFilterPreAnalysis joinFilterPreAnalysis; /** * @param baseSegment The left-hand side base segment @@ -53,14 +53,12 @@ public class HashJoinSegment extends AbstractSegment public HashJoinSegment( Segment baseSegment, List clauses, - boolean enableFilterPushDown, - boolean enableFilterRewrite + JoinFilterPreAnalysis joinFilterPreAnalysis ) { this.baseSegment = baseSegment; this.clauses = clauses; - this.enableFilterPushDown = enableFilterPushDown; - this.enableFilterRewrite = enableFilterRewrite; + this.joinFilterPreAnalysis = joinFilterPreAnalysis; // Verify 'clauses' is nonempty (otherwise it's a waste to create this object, and the caller should know) if (clauses.isEmpty()) { @@ -93,7 +91,7 @@ public QueryableIndex asQueryableIndex() @Override public StorageAdapter asStorageAdapter() { - return new HashJoinSegmentStorageAdapter(baseSegment.asStorageAdapter(), clauses, enableFilterPushDown, enableFilterRewrite); + return new HashJoinSegmentStorageAdapter(baseSegment.asStorageAdapter(), clauses, joinFilterPreAnalysis); } @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 f59009953309..5265478e68bd 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 @@ -19,13 +19,11 @@ package org.apache.druid.segment.join; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; 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; -import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.Capabilities; @@ -38,6 +36,7 @@ import org.apache.druid.segment.data.Indexed; 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.joda.time.DateTime; import org.joda.time.Interval; @@ -55,40 +54,22 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter { private final StorageAdapter baseAdapter; private final List clauses; - private final boolean enableFilterPushDown; - private final boolean enableFilterRewrite; + private final JoinFilterPreAnalysis joinFilterPreAnalysis; /** * @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 enableFilterPushDown Whether to enable filter push down optimizations to the base segment */ HashJoinSegmentStorageAdapter( StorageAdapter baseAdapter, List clauses, - final boolean enableFilterPushDown, - final boolean enableFilterRewrite + final JoinFilterPreAnalysis joinFilterPreAnalysis ) { this.baseAdapter = baseAdapter; this.clauses = clauses; - this.enableFilterPushDown = enableFilterPushDown; - this.enableFilterRewrite = enableFilterRewrite; - } - - @VisibleForTesting - HashJoinSegmentStorageAdapter( - StorageAdapter baseAdapter, - List clauses - ) - { - this( - baseAdapter, - clauses, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE - ); + this.joinFilterPreAnalysis = joinFilterPreAnalysis; } @Override @@ -237,22 +218,16 @@ public Sequence makeCursors( @Nullable final QueryMetrics queryMetrics ) { - final List preJoinVirtualColumns = new ArrayList<>(); final List postJoinVirtualColumns = new ArrayList<>(); + final Set baseColumns = determineBaseColumnsWithPreAndPostJoinVirtualColumns( virtualColumns, preJoinVirtualColumns, postJoinVirtualColumns ); - JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter( - this, - baseColumns, - filter, - enableFilterPushDown, - enableFilterRewrite - ); + JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); 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 @@ -301,10 +276,12 @@ public boolean isBaseColumn(final String column) return !getClauseForColumn(column).isPresent(); } + /* public boolean isEnableFilterPushDown() { return enableFilterPushDown; } + */ /** * Return a String set containing the name of columns that belong to the base table (including any pre-join virtual diff --git a/processing/src/main/java/org/apache/druid/segment/join/Joinable.java b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java index 63c58b44031d..a9ada797599c 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/Joinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java @@ -89,6 +89,8 @@ JoinMatcher makeJoinMatcher( Set getCorrelatedColumnValues( String searchColumnName, String searchColumnValue, - String retrievalColumnName + String retrievalColumnName, + long maxCorrelationSetSize, + boolean allowNonKeyColumnSearch ); } 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 792eed0420e6..5cf2bfb0e7ab 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 @@ -21,9 +21,13 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.filter.Filter; import org.apache.druid.query.planning.PreJoinableClause; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; +import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; @@ -83,9 +87,14 @@ public static Function createSegmentMapFn( final JoinableFactory joinableFactory, final AtomicLong cpuTimeAccumulator, final boolean enableFilterPushDown, - final boolean enableFilterRewrite + final boolean enableFilterRewrite, + final boolean enableRewriteValueColumnFilters, + final long filterRewriteMaxSize, + final Filter originalFilter, + final VirtualColumns virtualColumns ) { + // compute column correlations here and RHS correlated values return JvmUtils.safeAccumulateThreadCpuTime( cpuTimeAccumulator, () -> { @@ -93,7 +102,16 @@ public static Function createSegmentMapFn( return Function.identity(); } else { final List joinableClauses = createJoinableClauses(clauses, joinableFactory); - return baseSegment -> new HashJoinSegment(baseSegment, joinableClauses, enableFilterPushDown, enableFilterRewrite); + JoinFilterPreAnalysis jfpa = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + virtualColumns, + originalFilter, + enableFilterPushDown, + enableFilterRewrite, + enableRewriteValueColumnFilters, + filterRewriteMaxSize + ); + return baseSegment -> new HashJoinSegment(baseSegment, joinableClauses, jfpa); } } ); 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 574446decfaf..4f8bdcd73357 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 @@ -20,25 +20,26 @@ package org.apache.druid.segment.join.filter; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.math.expr.Expr; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.filter.AndFilter; 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.Equality; -import org.apache.druid.segment.join.HashJoinSegmentStorageAdapter; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinableClause; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; -import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -50,22 +51,29 @@ /** * When there is a filter in a join query, we can sometimes improve performance by applying parts of the filter * when we first read from the base table instead of after the join. - * - * This class provides a {@link #splitFilter(HashJoinSegmentStorageAdapter, Set, Filter, boolean, boolean)} method that - * takes a filter and splits it into a portion that should be applied to the base table prior to the join, and a - * portion that should be applied after the join. - * + *

* The first step of the filter splitting is to convert the filter into * https://en.wikipedia.org/wiki/Conjunctive_normal_form (an AND of ORs). This allows us to consider each * OR clause independently as a candidate for filter push down to the base table. - * + *

* A filter clause can be pushed down if it meets one of the following conditions: * - The filter only applies to columns from the base table * - The filter applies to columns from the join table, and we determine that the filter can be rewritten * into a filter on columns from the base table - * + *

* For the second case, where we rewrite filter clauses, the rewritten clause can be less selective than the original, * so we preserve the original clause in the post-join filtering phase. + *

+ * The starting point for join analysis is the {@link #computeJoinFilterPreAnalysis} method. This method should be + * called before performing any per-segment join query work. This method converts the query filter into + * conjunctive normal form, and splits the CNF clauses into a portion that only references base table columns and + * a portion that references join table columns. For the filter clauses that apply to join table columns, the + * pre-analysis step computes the information necessary for rewriting such filters into filters on base table columns. + *

+ * The result of this pre-analysis method should be passed into the next step of join filter analysis, described below. + *

+ * The {@link #splitFilter(JoinFilterPreAnalysis)} method takes the pre-analysis result and optionally applies the\ + * filter rewrite and push down operations on a per-segment level. */ public class JoinFilterAnalyzer { @@ -73,50 +81,96 @@ public class JoinFilterAnalyzer private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory(); /** - * Analyze a filter and return a JoinFilterSplit indicating what parts of the filter should be applied pre-join - * and post-join. + * Before making per-segment filter splitting decisions, we first do a pre-analysis step + * where we convert the query filter (if any) into conjunctive normal form and then + * determine the structure of RHS filter rewrites (if any), since this information is shared across all + * per-segment operations. * - * @param hashJoinSegmentStorageAdapter The storage adapter that is being queried - * @param baseColumnNames Set of names of columns that belong to the base table, - * including pre-join virtual columns - * @param originalFilter Original filter from the query - * @param enableFilterPushDown Whether to enable filter push down - * @return A JoinFilterSplit indicating what parts of the filter should be applied pre-join - * and post-join. + * See {@link JoinFilterPreAnalysis} for details on the result of this pre-analysis step. + * + * @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. + * + * @return A JoinFilterPreAnalysis containing information determined in this pre-analysis step. */ - public static JoinFilterSplit splitFilter( - HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter, - Set baseColumnNames, - @Nullable Filter originalFilter, + public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( + List joinableClauses, + VirtualColumns virtualColumns, + Filter originalFilter, boolean enableFilterPushDown, - boolean enableFilterRewrite + boolean enableFilterRewrite, + boolean enableRewriteValueColumnFilters, + long filterRewriteMaxSize ) { - if (originalFilter == null) { - return new JoinFilterSplit( + final List preJoinVirtualColumns = new ArrayList<>(); + final List postJoinVirtualColumns = new ArrayList<>(); + + splitVirtualColumns(joinableClauses, virtualColumns, preJoinVirtualColumns, postJoinVirtualColumns); + + if (originalFilter == null || !enableFilterPushDown) { + return new JoinFilterPreAnalysis( + joinableClauses, + originalFilter, + postJoinVirtualColumns, null, null, - ImmutableList.of() + null, + enableFilterPushDown, + enableFilterRewrite ); } - if (!enableFilterPushDown) { - return new JoinFilterSplit( - null, + Filter normalizedFilter = Filters.convertToCNF(originalFilter); + + // List of candidates for pushdown + // CNF normalization will generate either + // - an AND filter with multiple subfilters + // - or a single non-AND subfilter which cannot be split further + List normalizedOrClauses; + if (normalizedFilter instanceof AndFilter) { + normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters(); + } else { + normalizedOrClauses = Collections.singletonList(normalizedFilter); + } + + List normalizedBaseTableClauses = new ArrayList<>(); + List normalizedJoinTableClauses = new ArrayList<>(); + + for (Filter orClause : normalizedOrClauses) { + Set reqColumns = orClause.getRequiredColumns(); + if (areSomeColumnsFromJoin(joinableClauses, reqColumns) || areSomeColumnsFromPostJoinVirtualColumns( + postJoinVirtualColumns, + reqColumns + )) { + normalizedJoinTableClauses.add(orClause); + } else { + normalizedBaseTableClauses.add(orClause); + } + } + + if (!enableFilterRewrite) { + return new JoinFilterPreAnalysis( + joinableClauses, originalFilter, - ImmutableList.of() + postJoinVirtualColumns, + normalizedBaseTableClauses, + normalizedJoinTableClauses, + null, + enableFilterPushDown, + enableFilterRewrite ); } - Filter normalizedFilter = Filters.convertToCNF(originalFilter); - - // build the prefix and equicondition maps - // We should check that the prefixes do not duplicate or shadow each other. This is not currently implemented, - // but this is tracked at https://github.com/apache/druid/issues/9329 + // build the equicondition map, used for determining how the tables are connected through joins Map> equiconditions = new HashMap<>(); - Map prefixes = new HashMap<>(); - for (JoinableClause clause : hashJoinSegmentStorageAdapter.getClauses()) { - prefixes.put(clause.getPrefix(), clause); + for (JoinableClause clause : joinableClauses) { for (Equality equality : clause.getCondition().getEquiConditions()) { Set exprsForRhs = equiconditions.computeIfAbsent( clause.getPrefix() + equality.getRightColumn(), @@ -128,31 +182,168 @@ public static JoinFilterSplit splitFilter( } } - // List of candidates for pushdown - // CNF normalization will generate either - // - an AND filter with multiple subfilters - // - or a single non-AND subfilter which cannot be split further - List normalizedOrClauses; - if (normalizedFilter instanceof AndFilter) { - normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters(); - } else { - normalizedOrClauses = Collections.singletonList(normalizedFilter); + List pushDownVirtualColumns = new ArrayList<>(); + Map>> correlationsByPrefix = new HashMap<>(); + + // Determine candidates for filter rewrites. + // A candidate is an RHS column that appears in a filter, along with the value being filtered on, plus + // the joinable clause associated with the table that the RHS column is from. + Set rhsRewriteCandidates = new HashSet<>(); + for (Filter orClause : normalizedJoinTableClauses) { + if (filterMatchesNull(orClause)) { + continue; + } + + if (orClause instanceof SelectorFilter) { + // this is a candidate for RHS filter rewrite, determine column correlations and correlated values + String reqColumn = ((SelectorFilter) orClause).getDimension(); + String reqValue = ((SelectorFilter) orClause).getValue(); + JoinableClause joinableClause = isColumnFromJoin(joinableClauses, reqColumn); + if (joinableClause != null) { + rhsRewriteCandidates.add( + new RHSRewriteCandidate( + joinableClause, + reqColumn, + reqValue + ) + ); + } + } + + if (orClause instanceof OrFilter) { + for (Filter subFilter : ((OrFilter) orClause).getFilters()) { + if (subFilter instanceof SelectorFilter) { + String reqColumn = ((SelectorFilter) subFilter).getDimension(); + String reqValue = ((SelectorFilter) subFilter).getValue(); + JoinableClause joinableClause = isColumnFromJoin(joinableClauses, reqColumn); + if (joinableClause != null) { + rhsRewriteCandidates.add( + new RHSRewriteCandidate( + joinableClause, + reqColumn, + reqValue + ) + ); + } + } + } + } + } + + // Build a map of RHS table prefix -> JoinFilterColumnCorrelationAnalysis based on the RHS rewrite candidates + for (RHSRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates) { + Optional> correlationsForPrefix = correlationsByPrefix.computeIfAbsent( + rhsRewriteCandidate.getJoinableClause().getPrefix(), + p -> findCorrelatedBaseTableColumns( + joinableClauses, + p, + rhsRewriteCandidate.getJoinableClause(), + equiconditions + ) + ); + } + + // Using the RHS table prefix -> JoinFilterColumnCorrelationAnalysis created in the previous step, + // build a map of rhsFilterColumn -> Pair(rhsFilterColumn, rhsFilterValue) -> correlatedValues for specific filter pair + // The Pair(rhsFilterColumn, rhsFilterValue) -> correlatedValues mappings are stored in the + // JoinFilterColumnCorrelationAnalysis objects, which are shared across all rhsFilterColumn entries that belong + // to the same RHS table. + // + // The value is a List instead of a single value because a table can be joined + // to another via multiple columns. + // (See JoinFilterAnalyzerTest.test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnRHS for an example) + Map>> correlationsByFilteringColumn = new HashMap<>(); + for (RHSRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates) { + Optional> correlationsForPrefix = correlationsByPrefix.get( + rhsRewriteCandidate.getJoinableClause().getPrefix() + ); + if (correlationsForPrefix.isPresent()) { + for (Map.Entry correlationForPrefix : correlationsForPrefix.get() + .entrySet()) { + Optional> perColumnCorrelations = + correlationsByFilteringColumn.computeIfAbsent( + rhsRewriteCandidate.getRhsColumn(), + (rhsCol) -> { + return Optional.of(new ArrayList<>()); + } + ); + perColumnCorrelations.get().add(correlationForPrefix.getValue()); + correlationForPrefix.getValue().getCorrelatedValuesMap().computeIfAbsent( + Pair.of(rhsRewriteCandidate.getRhsColumn(), rhsRewriteCandidate.getValueForRewrite()), + (rhsVal) -> { + Set correlatedValues = getCorrelatedValuesForPushDown( + rhsRewriteCandidate.getRhsColumn(), + rhsRewriteCandidate.getValueForRewrite(), + correlationForPrefix.getValue().getJoinColumn(), + rhsRewriteCandidate.getJoinableClause(), + enableRewriteValueColumnFilters, + filterRewriteMaxSize + ); + + if (correlatedValues.isEmpty()) { + return Optional.empty(); + } else { + return Optional.of(correlatedValues); + } + } + ); + } + } else { + correlationsByFilteringColumn.put(rhsRewriteCandidate.getRhsColumn(), Optional.empty()); + } + } + + // Go through each per-column analysis list and prune duplicates + for (Map.Entry>> correlation : correlationsByFilteringColumn.entrySet()) { + if (correlation.getValue().isPresent()) { + List dedupList = eliminateCorrelationDuplicates( + correlation.getValue().get() + ); + correlationsByFilteringColumn.put(correlation.getKey(), Optional.of(dedupList)); + } + } + + return new JoinFilterPreAnalysis( + joinableClauses, + originalFilter, + postJoinVirtualColumns, + normalizedBaseTableClauses, + normalizedJoinTableClauses, + correlationsByFilteringColumn, + enableFilterPushDown, + enableFilterRewrite + ); + } + + public static JoinFilterSplit splitFilter( + JoinFilterPreAnalysis joinFilterPreAnalysis + ) + { + if (joinFilterPreAnalysis.getOriginalFilter() == null || !joinFilterPreAnalysis.isEnableFilterPushDown()) { + return new JoinFilterSplit( + null, + joinFilterPreAnalysis.getOriginalFilter(), + ImmutableList.of() + ); } // Pushdown filters, rewriting if necessary List leftFilters = new ArrayList<>(); List rightFilters = new ArrayList<>(); List pushDownVirtualColumns = new ArrayList<>(); - Map>> correlationCache = new HashMap<>(); - for (Filter orClause : normalizedOrClauses) { + for (Filter baseTableFilter : joinFilterPreAnalysis.getNormalizedBaseTableClauses()) { + if (!filterMatchesNull(baseTableFilter)) { + leftFilters.add(baseTableFilter); + } else { + rightFilters.add(baseTableFilter); + } + } + + for (Filter orClause : joinFilterPreAnalysis.getNormalizedJoinTableClauses()) { JoinFilterAnalysis joinFilterAnalysis = analyzeJoinFilterClause( - baseColumnNames, orClause, - prefixes, - equiconditions, - correlationCache, - enableFilterRewrite + joinFilterPreAnalysis ); if (joinFilterAnalysis.isCanPushDown()) { leftFilters.add(joinFilterAnalysis.getPushDownFilter().get()); @@ -173,133 +364,62 @@ public static JoinFilterSplit splitFilter( } - - /** - * Analyze a filter clause from a filter that is in conjunctive normal form (AND of ORs). - * The clause is expected to be an OR filter or a leaf filter. - * - * @param baseColumnNames Set of names of columns that belong to the base table, including pre-join virtual columns - * @param filterClause Individual filter clause (an OR filter or a leaf filter) from a filter that is in CNF - * @param prefixes Map of table prefixes - * @param equiconditions Equicondition map - * @param correlationCache Cache of column correlation analyses. - * - * @return a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter. - */ private static JoinFilterAnalysis analyzeJoinFilterClause( - Set baseColumnNames, Filter filterClause, - Map prefixes, - Map> equiconditions, - Map>> correlationCache, - boolean enableFilterRewrite - + JoinFilterPreAnalysis joinFilterPreAnalysis ) { // NULL matching conditions are not currently pushed down. // They require special consideration based on the join type, and for simplicity of the initial implementation // this is not currently handled. - if (filterMatchesNull(filterClause)) { - return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause); - } - - boolean baseTableOnly = true; - for (String requiredColumn : filterClause.getRequiredColumns()) { - if (!baseColumnNames.contains(requiredColumn)) { - baseTableOnly = false; - break; - } - } - - if (baseTableOnly) { - return new JoinFilterAnalysis( - false, - filterClause, - filterClause, - ImmutableList.of() - ); - } - - if (!enableFilterRewrite) { + if (!joinFilterPreAnalysis.isEnableFilterRewrite() || filterMatchesNull(filterClause)) { return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause); } // Currently we only support rewrites of selector filters and selector filters within OR filters. if (filterClause instanceof SelectorFilter) { return rewriteSelectorFilter( - baseColumnNames, (SelectorFilter) filterClause, - prefixes, - equiconditions, - correlationCache + joinFilterPreAnalysis ); } if (filterClause instanceof OrFilter) { return rewriteOrFilter( - baseColumnNames, (OrFilter) filterClause, - prefixes, - equiconditions, - correlationCache + joinFilterPreAnalysis ); } return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause); } - /** - * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to - * the base table. - * - * @param baseColumnNames Set of names of columns that belong to the base table, including pre-join virtual columns - * @param orFilter OrFilter to be rewritten - * @param prefixes Map of table prefixes to clauses - * @param equiconditions Map of equiconditions - * @param correlationCache Column correlation analysis cache. This will be potentially modified by adding - * any new column correlation analyses to the cache. - * - * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter - */ private static JoinFilterAnalysis rewriteOrFilter( - Set baseColumnNames, OrFilter orFilter, - Map prefixes, - Map> equiconditions, - Map>> correlationCache + JoinFilterPreAnalysis joinFilterPreAnalysis ) { boolean retainRhs = false; - List newFilters = new ArrayList<>(); for (Filter filter : orFilter.getFilters()) { - boolean allBaseColumns = true; - for (String requiredColumn : filter.getRequiredColumns()) { - if (!baseColumnNames.contains(requiredColumn)) { - allBaseColumns = false; - } + if (!areSomeColumnsFromJoin(joinFilterPreAnalysis.getJoinableClauses(), filter.getRequiredColumns())) { + newFilters.add(filter); + continue; } - if (!allBaseColumns) { - retainRhs = true; - if (filter instanceof SelectorFilter) { - JoinFilterAnalysis rewritten = rewriteSelectorFilter( - baseColumnNames, - (SelectorFilter) filter, - prefixes, - equiconditions, - correlationCache - ); - if (!rewritten.isCanPushDown()) { - return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter); - } else { - newFilters.add(rewritten.getPushDownFilter().get()); - } - } else { + retainRhs = true; + if (filter instanceof SelectorFilter) { + JoinFilterAnalysis rewritten = rewriteSelectorFilter( + (SelectorFilter) filter, + joinFilterPreAnalysis + ); + if (!rewritten.isCanPushDown()) { return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter); + } else { + newFilters.add(rewritten.getPushDownFilter().get()); } } else { - newFilters.add(filter); + return JoinFilterAnalysis.createNoPushdownFilterAnalysis(orFilter); } } @@ -311,124 +431,99 @@ private static JoinFilterAnalysis rewriteOrFilter( ); } - /** - * Rewrites a selector filter on a join table into an IN filter on the base table. - * - * @param baseColumnNames Set of names of columns that belong to the base table, including pre-join virtual - * columns - * @param selectorFilter SelectorFilter to be rewritten - * @param prefixes Map of join table prefixes to clauses - * @param equiconditions Map of equiconditions - * @param correlationCache Cache of column correlation analyses. This will be potentially modified by adding - * any new column correlation analyses to the cache. - * - * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter - */ private static JoinFilterAnalysis rewriteSelectorFilter( - Set baseColumnNames, SelectorFilter selectorFilter, - Map prefixes, - Map> equiconditions, - Map>> correlationCache + JoinFilterPreAnalysis joinFilterPreAnalysis ) { + + List newFilters = new ArrayList<>(); + List pushdownVirtualColumns = new ArrayList<>(); + String filteringColumn = selectorFilter.getDimension(); - for (Map.Entry prefixAndClause : prefixes.entrySet()) { - if (prefixAndClause.getValue().includesColumn(filteringColumn)) { - Optional> correlations = correlationCache.computeIfAbsent( - prefixAndClause.getKey(), - p -> findCorrelatedBaseTableColumns( - baseColumnNames, - p, - prefixes.get(p), - equiconditions - ) - ); + String filteringValue = selectorFilter.getValue(); - if (!correlations.isPresent()) { - return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter); - } + if (areSomeColumnsFromPostJoinVirtualColumns( + joinFilterPreAnalysis.getPostJoinVirtualColumns(), + selectorFilter.getRequiredColumns() + )) { + return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter); + } - List newFilters = new ArrayList<>(); - List pushdownVirtualColumns = new ArrayList<>(); + if (!areSomeColumnsFromJoin(joinFilterPreAnalysis.getJoinableClauses(), selectorFilter.getRequiredColumns())) { + return new JoinFilterAnalysis( + true, + selectorFilter, + selectorFilter, + pushdownVirtualColumns + ); + } - for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlations.get()) { - if (correlationAnalysis.supportsPushDown()) { - Set correlatedValues = getCorrelatedValuesForPushDown( - selectorFilter.getDimension(), - selectorFilter.getValue(), - correlationAnalysis.getJoinColumn(), - prefixAndClause.getValue() - ); + Optional> correlationAnalyses = joinFilterPreAnalysis.getCorrelationsByFilteringColumn() + .get(filteringColumn); - if (correlatedValues.isEmpty()) { - return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter); - } + if (!correlationAnalyses.isPresent()) { + return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter); + } - for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) { - Filter rewrittenFilter = new InDimFilter( - correlatedBaseColumn, - correlatedValues, - null, - null - ).toFilter(); - newFilters.add(rewrittenFilter); - } - for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) { - // We need to create a virtual column for the expressions when pushing down. - // Note that this block is never entered right now, since correlationAnalysis.supportsPushDown() - // will return false if there any correlated expressions on the base table. - // Pushdown of such filters is disabled until the expressions system supports converting an expression - // into a String representation that can be reparsed into the same expression. - // https://github.com/apache/druid/issues/9326 tracks this expressions issue. - String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size()); - - VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn( - vcName, - correlatedBaseExpr, - ValueType.STRING - ); - pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn); - - Filter rewrittenFilter = new InDimFilter( - vcName, - correlatedValues, - null, - null - ).toFilter(); - newFilters.add(rewrittenFilter); - } - } - } + for (JoinFilterColumnCorrelationAnalysis correlationAnalysis : correlationAnalyses.get()) { + if (correlationAnalysis.supportsPushDown()) { + Optional> correlatedValues = correlationAnalysis.getCorrelatedValuesMap().get( + Pair.of(filteringColumn, filteringValue) + ); - if (newFilters.isEmpty()) { + if (!correlatedValues.isPresent()) { return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter); } - return new JoinFilterAnalysis( - true, - selectorFilter, - Filters.and(newFilters), - pushdownVirtualColumns - ); + for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) { + Filter rewrittenFilter = new InDimFilter( + correlatedBaseColumn, + correlatedValues.get(), + null, + null + ).toFilter(); + newFilters.add(rewrittenFilter); + } + + for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) { + // We need to create a virtual column for the expressions when pushing down. + // Note that this block is never entered right now, since correlationAnalysis.supportsPushDown() + // will return false if there any correlated expressions on the base table. + // Pushdown of such filters is disabled until the expressions system supports converting an expression + // into a String representation that can be reparsed into the same expression. + // https://github.com/apache/druid/issues/9326 tracks this expressions issue. + String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size()); + + VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn( + vcName, + correlatedBaseExpr, + ValueType.STRING + ); + pushdownVirtualColumns.add(correlatedBaseExprVirtualColumn); + + Filter rewrittenFilter = new InDimFilter( + vcName, + correlatedValues.get(), + null, + null + ).toFilter(); + newFilters.add(rewrittenFilter); + } } } - // We're not filtering directly on a column from one of the join tables, but - // we might be filtering on a post-join virtual column (which won't have a join prefix). We cannot - // push down such filters, so check that the filtering column appears in the set of base column names (which - // includes pre-join virtual columns). - if (baseColumnNames.contains(filteringColumn)) { - return new JoinFilterAnalysis( - false, - selectorFilter, - selectorFilter, - ImmutableList.of() - ); - } else { + if (newFilters.isEmpty()) { return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter); } + + return new JoinFilterAnalysis( + true, + selectorFilter, + Filters.and(newFilters), + pushdownVirtualColumns + ); } private static String getCorrelatedBaseExprVirtualColumnName(int counter) @@ -453,7 +548,9 @@ private static Set getCorrelatedValuesForPushDown( String filterColumn, String filterValue, String correlatedJoinColumn, - JoinableClause clauseForFilteredTable + JoinableClause clauseForFilteredTable, + boolean enableRewriteValueColumnFilters, + long filterRewriteMaxSize ) { String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length()); @@ -462,45 +559,14 @@ private static Set getCorrelatedValuesForPushDown( return clauseForFilteredTable.getJoinable().getCorrelatedColumnValues( filterColumnNoPrefix, filterValue, - correlatedColumnNoPrefix + correlatedColumnNoPrefix, + filterRewriteMaxSize, + enableRewriteValueColumnFilters ); } - /** - * For each rhs column that appears in the equiconditions for a table's JoinableClause, - * we try to determine what base table columns are related to the rhs column through the total set of equiconditions. - * We do this by searching backwards through the chain of join equiconditions using the provided equicondition map. - * - * For example, suppose we have 3 tables, A,B,C, joined with the following conditions, where A is the base table: - * A.joinColumn == B.joinColumn - * B.joinColum == C.joinColumn - * - * We would determine that C.joinColumn is correlated with A.joinColumn: we first see that - * C.joinColumn is linked to B.joinColumn which in turn is linked to A.joinColumn - * - * Suppose we had the following join conditions instead: - * f(A.joinColumn) == B.joinColumn - * B.joinColum == C.joinColumn - * In this case, the JoinFilterColumnCorrelationAnalysis for C.joinColumn would be linked to f(A.joinColumn). - * - * Suppose we had the following join conditions instead: - * A.joinColumn == B.joinColumn - * f(B.joinColum) == C.joinColumn - * - * Because we cannot reverse the function f() applied to the second table B in all cases, - * we cannot relate C.joinColumn to A.joinColumn, and we would not generate a correlation for C.joinColumn - * - * @param baseColumnNames Set of names of columns that belong to the base table, including pre-join virtual - * columns - * @param tablePrefix Prefix for a join table - * @param clauseForTablePrefix Joinable clause for the prefix - * @param equiConditions Map of equiconditions, keyed by the right hand columns - * - * @return A list of correlatation analyses for the equicondition RHS columns that reside in the table associated with - * the tablePrefix - */ - private static Optional> findCorrelatedBaseTableColumns( - Set baseColumnNames, + private static Optional> findCorrelatedBaseTableColumns( + List joinableClauses, String tablePrefix, JoinableClause clauseForTablePrefix, Map> equiConditions @@ -513,14 +579,14 @@ private static Optional> findCorrelate rhsColumns.add(tablePrefix + eq.getRightColumn()); } - List correlations = new ArrayList<>(); + Map correlations = new HashMap<>(); for (String rhsColumn : rhsColumns) { Set correlatedBaseColumns = new HashSet<>(); Set correlatedBaseExpressions = new HashSet<>(); getCorrelationForRHSColumn( - baseColumnNames, + joinableClauses, equiConditions, rhsColumn, correlatedBaseColumns, @@ -528,10 +594,11 @@ private static Optional> findCorrelate ); if (correlatedBaseColumns.isEmpty() && correlatedBaseExpressions.isEmpty()) { - return Optional.empty(); + continue; } - correlations.add( + correlations.put( + rhsColumn, new JoinFilterColumnCorrelationAnalysis( rhsColumn, correlatedBaseColumns, @@ -540,25 +607,15 @@ private static Optional> findCorrelate ); } - List dedupCorrelations = eliminateCorrelationDuplicates(correlations); - - return Optional.of(dedupCorrelations); + if (correlations.size() == 0) { + return Optional.empty(); + } else { + return Optional.of(correlations); + } } - /** - * Helper method for {@link #findCorrelatedBaseTableColumns} that determines correlated base table columns - * and/or expressions for a single RHS column and adds them to the provided sets as it traverses the - * equicondition column relationships. - * - * @param baseColumnNames Set of names of columns that belong to the base table, including pre-join virtual columns - * @param equiConditions Map of equiconditions, keyed by the right hand columns - * @param rhsColumn RHS column to find base table correlations for - * @param correlatedBaseColumns Set of correlated base column names for the provided RHS column. Will be modified. - * @param correlatedBaseExpressions Set of correlated base column expressions for the provided RHS column. Will be - * modified. - */ private static void getCorrelationForRHSColumn( - Set baseColumnNames, + List joinableClauses, Map> equiConditions, String rhsColumn, Set correlatedBaseColumns, @@ -577,18 +634,19 @@ private static void getCorrelationForRHSColumn( // We push down if the function only requires base table columns Expr.BindingDetails bindingDetails = lhsExpr.analyzeInputs(); Set requiredBindings = bindingDetails.getRequiredBindings(); - if (!baseColumnNames.containsAll(requiredBindings)) { + + if (areSomeColumnsFromJoin(joinableClauses, requiredBindings)) { break; } correlatedBaseExpressions.add(lhsExpr); } else { // simple identifier, see if we can correlate it with a column on the base table findMappingFor = identifier; - if (baseColumnNames.contains(identifier)) { + if (isColumnFromJoin(joinableClauses, identifier) == null) { correlatedBaseColumns.add(findMappingFor); } else { getCorrelationForRHSColumn( - baseColumnNames, + joinableClauses, equiConditions, findMappingFor, correlatedBaseColumns, @@ -602,20 +660,21 @@ private static void getCorrelationForRHSColumn( /** * Given a list of JoinFilterColumnCorrelationAnalysis, prune the list so that we only have one * JoinFilterColumnCorrelationAnalysis for each unique combination of base columns. - * + *

* Suppose we have a join condition like the following, where A is the base table: - * A.joinColumn == B.joinColumn && A.joinColumn == B.joinColumn2 - * + * A.joinColumn == B.joinColumn && A.joinColumn == B.joinColumn2 + *

* We only need to consider one correlation to A.joinColumn since B.joinColumn and B.joinColumn2 must * have the same value in any row that matches the join condition. - * + *

* In the future this method could consider which column correlation should be preserved based on availability of * indices and other heuristics. - * + *

* When push down of filters with LHS expressions in the join condition is supported, this method should also * consider expressions. * * @param originalList Original list of column correlation analyses. + * * @return Pruned list of column correlation analyses. */ private static List eliminateCorrelationDuplicates( @@ -635,4 +694,106 @@ private static boolean filterMatchesNull(Filter filter) ValueMatcher valueMatcher = filter.makeMatcher(ALL_NULL_COLUMN_SELECTOR_FACTORY); return valueMatcher.matches(); } + + private static JoinableClause isColumnFromJoin( + List joinableClauses, + String column + ) + { + for (JoinableClause joinableClause : joinableClauses) { + if (joinableClause.includesColumn(column)) { + return joinableClause; + } + } + + return null; + } + + private static boolean isColumnFromPostJoinVirtualColumns( + List postJoinVirtualColumns, + String column + ) + { + for (VirtualColumn postJoinVirtualColumn : postJoinVirtualColumns) { + if (column.equals(postJoinVirtualColumn.getOutputName())) { + return true; + } + } + return false; + } + + private static boolean areSomeColumnsFromJoin( + List joinableClauses, + Collection columns + ) + { + for (String column : columns) { + if (isColumnFromJoin(joinableClauses, column) != null) { + return true; + } + } + return false; + } + + private static boolean areSomeColumnsFromPostJoinVirtualColumns( + List postJoinVirtualColumns, + Collection columns + ) + { + for (String column : columns) { + if (isColumnFromPostJoinVirtualColumns(postJoinVirtualColumns, column)) { + return true; + } + } + return false; + } + + private static void splitVirtualColumns( + List joinableClauses, + final VirtualColumns virtualColumns, + final List preJoinVirtualColumns, + final List postJoinVirtualColumns + ) + { + for (VirtualColumn virtualColumn : virtualColumns.getVirtualColumns()) { + if (areSomeColumnsFromJoin(joinableClauses, virtualColumn.requiredColumns())) { + postJoinVirtualColumns.add(virtualColumn); + } else { + preJoinVirtualColumns.add(virtualColumn); + } + } + } + + private static class RHSRewriteCandidate + { + private final JoinableClause joinableClause; + private final String rhsColumn; + private final String valueForRewrite; + + public RHSRewriteCandidate( + JoinableClause joinableClause, + String rhsColumn, + String valueForRewrite + ) + { + this.joinableClause = joinableClause; + this.rhsColumn = rhsColumn; + this.valueForRewrite = valueForRewrite; + } + + public JoinableClause getJoinableClause() + { + return joinableClause; + } + + public String getRhsColumn() + { + return rhsColumn; + } + + public String getValueForRewrite() + { + return valueForRewrite; + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java index c27d13c86834..57efeb5c8399 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java @@ -19,10 +19,14 @@ package org.apache.druid.segment.join.filter; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.math.expr.Expr; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; /** @@ -37,6 +41,7 @@ public class JoinFilterColumnCorrelationAnalysis private final String joinColumn; private final List baseColumns; private final List baseExpressions; + private Map, Optional>> correlatedValuesMap; public JoinFilterColumnCorrelationAnalysis( String joinColumn, @@ -48,6 +53,7 @@ public JoinFilterColumnCorrelationAnalysis( this.baseColumns = new ArrayList<>(baseColumns); this.baseExpressions = new ArrayList<>(baseExpressions); this.baseColumns.sort(String.CASE_INSENSITIVE_ORDER); + this.correlatedValuesMap = new HashMap<>(); } public String getJoinColumn() @@ -65,6 +71,11 @@ public List getBaseExpressions() return baseExpressions; } + public Map, Optional>> getCorrelatedValuesMap() + { + return correlatedValuesMap; + } + public boolean supportsPushDown() { return !baseColumns.isEmpty() && baseExpressions.isEmpty(); 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 new file mode 100644 index 000000000000..a2ec5b911fe9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java @@ -0,0 +1,113 @@ +/* + * 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; + +import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.join.JoinableClause; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * 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) + * - 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 mapping of RHS filtering columns -> List, used for filter rewrites + * - 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. + */ +public class JoinFilterPreAnalysis +{ + private final List joinableClauses; + private final Filter originalFilter; + private final List normalizedBaseTableClauses; + private final List normalizedJoinTableClauses; + private final Map>> correlationsByFilteringColumn; + private final boolean enableFilterPushDown; + private final boolean enableFilterRewrite; + private final List postJoinVirtualColumns; + + public JoinFilterPreAnalysis( + final List joinableClauses, + final Filter originalFilter, + final List postJoinVirtualColumns, + final List normalizedBaseTableClauses, + final List normalizedJoinTableClauses, + final Map>> correlationsByFilteringColumn, + final boolean enableFilterPushDown, + final boolean enableFilterRewrite + ) + { + this.joinableClauses = joinableClauses; + this.originalFilter = originalFilter; + this.postJoinVirtualColumns = postJoinVirtualColumns; + this.normalizedBaseTableClauses = normalizedBaseTableClauses; + this.normalizedJoinTableClauses = normalizedJoinTableClauses; + this.correlationsByFilteringColumn = correlationsByFilteringColumn; + this.enableFilterPushDown = enableFilterPushDown; + this.enableFilterRewrite = enableFilterRewrite; + } + + public List getJoinableClauses() + { + return joinableClauses; + } + + public Filter getOriginalFilter() + { + return originalFilter; + } + + public List getPostJoinVirtualColumns() + { + return postJoinVirtualColumns; + } + + public List getNormalizedBaseTableClauses() + { + return normalizedBaseTableClauses; + } + + public List getNormalizedJoinTableClauses() + { + return normalizedJoinTableClauses; + } + + public Map>> getCorrelationsByFilteringColumn() + { + return correlationsByFilteringColumn; + } + + public boolean isEnableFilterPushDown() + { + return enableFilterPushDown; + } + + public boolean isEnableFilterRewrite() + { + return enableFilterRewrite; + } +} + diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java index 1a739bc83f4c..6b1b41a762dd 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java @@ -90,7 +90,9 @@ public JoinMatcher makeJoinMatcher( public Set getCorrelatedColumnValues( String searchColumnName, String searchColumnValue, - String retrievalColumnName + String retrievalColumnName, + long maxCorrelationSetSize, + boolean allowNonKeyColumnSearch ) { Set correlatedValues; @@ -101,9 +103,14 @@ public Set getCorrelatedColumnValues( correlatedValues = ImmutableSet.of(extractor.apply(searchColumnName)); } } else { + if (!allowNonKeyColumnSearch) { + return ImmutableSet.of(); + } if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(retrievalColumnName)) { correlatedValues = ImmutableSet.of(searchColumnValue); } else { + // Lookup extractor unapply only provides a list of strings, so we can't respect + // maxCorrelationSetSize easily. This should be handled eventually. correlatedValues = ImmutableSet.copyOf(extractor.unapply(searchColumnValue)); } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java index 1eb01fe4e1ed..bb25890b7205 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java @@ -84,7 +84,9 @@ public JoinMatcher makeJoinMatcher( public Set getCorrelatedColumnValues( String searchColumnName, String searchColumnValue, - String retrievalColumnName + String retrievalColumnName, + long maxCorrelationSetSize, + boolean allowNonKeyColumnSearch ) { int filterColumnPosition = table.rowSignature().indexOf(searchColumnName); @@ -102,15 +104,26 @@ public Set getCorrelatedColumnValues( for (int i = 0; i < rowIndex.size(); i++) { int rowNum = rowIndex.getInt(i); correlatedValues.add(reader.read(rowNum).toString()); + + if (correlatedValues.size() > maxCorrelationSetSize) { + return ImmutableSet.of(); + } } return correlatedValues; } else { + if (!allowNonKeyColumnSearch) { + return ImmutableSet.of(); + } + IndexedTable.Reader dimNameReader = table.columnReader(filterColumnPosition); IndexedTable.Reader correlatedColumnReader = table.columnReader(correlatedColumnPosition); for (int i = 0; i < table.numRows(); i++) { if (searchColumnValue.equals(dimNameReader.read(i).toString())) { correlatedValues.add(correlatedColumnReader.read(i).toString()); } + if (correlatedValues.size() > maxCorrelationSetSize) { + return ImmutableSet.of(); + } } return correlatedValues; 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 6ebcf29a2d5f..9903269561a4 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 @@ -23,8 +23,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.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.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.segment.join.table.IndexedTableJoinable; @@ -182,9 +186,20 @@ protected JoinableClause regionToCountry(final JoinType joinType) protected HashJoinSegmentStorageAdapter makeFactToCountrySegment() { + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)), + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + return new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)) + ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)), + preAnalysis ); } 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 892376c9dcc5..c8460fe34108 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 @@ -27,13 +27,17 @@ 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; import org.apache.druid.query.filter.SelectorDimFilter; 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.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -41,6 +45,7 @@ import org.junit.Test; import java.util.Collections; +import java.util.List; public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorageAdapterTest { @@ -295,10 +300,23 @@ public void test_getMetadata_factToCountry() @Test public void test_makeCursors_factToCountryLeft() { + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)) + joinableClauses, + preAnalysis ).makeCursors( null, Intervals.ETERNITY, @@ -351,10 +369,22 @@ public void test_makeCursors_factToCountryLeft() @Test public void test_makeCursors_factToCountryInner() { + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.INNER)); + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.INNER)) + joinableClauses, + preAnalysis ).makeCursors( null, Intervals.ETERNITY, @@ -401,10 +431,22 @@ public void test_makeCursors_factToCountryInner() @Test public void test_makeCursors_factToCountryInnerUsingLookup() { + List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.INNER)); + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.INNER)) + joinableClauses, + preAnalysis ).makeCursors( null, Intervals.ETERNITY, @@ -452,13 +494,25 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumber() { // 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). + List joinableClauses = ImmutableList.of(factToCountryOnNumber(JoinType.INNER)); + Filter filter = new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(); + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnNumber(JoinType.INNER)) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -510,13 +564,25 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() { // 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). + List joinableClauses = ImmutableList.of(factToCountryNameUsingNumberLookup(JoinType.INNER)); + Filter filter = new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(); + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryNameUsingNumberLookup(JoinType.INNER)) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -564,12 +630,25 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() @Test public void test_makeCursors_factToCountryLeftWithFilterOnFacts() { + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); + Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -592,12 +671,25 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFacts() @Test public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() { + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); + Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("channel", null, null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -622,12 +714,25 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() @Test public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() { + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); + Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("channel", null, null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -652,16 +757,30 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() @Test public void test_makeCursors_factToCountryRightWithFilterOnJoinable() { + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); + Filter filter = new SelectorDimFilter( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + "Germany", + null + ).toFilter(); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", - "Germany", - null - ).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -685,16 +804,31 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinable() @Test public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() { + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); + + Filter filter = new OrDimFilter( + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", "DE", null), + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Norway", null), + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber", "10", null) + ).toFilter(); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)) + joinableClauses, + preAnalysis ).makeCursors( - new OrDimFilter( - new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", "DE", null), - new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Norway", null), - new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber", "10", null) - ).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -719,15 +853,29 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() @Test public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() { + List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); + Filter filter = new OrDimFilter( + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k", "DE", null), + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Norway", null) + ).toFilter(); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)) + joinableClauses, + preAnalysis ).makeCursors( - new OrDimFilter( - new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k", "DE", null), - new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Norway", null) - ).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -753,26 +901,41 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit // Join condition => always true. // Filter => Fact to countries on countryIsoCode. + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + "1", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + Filter filter = new ExpressionDimFilter( + StringUtils.format("\"%scountryIsoCode\" == countryIsoCode", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), + ExprMacroTable.nil() + ).toFilter(); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.INNER, - JoinConditionAnalysis.forExpression( - "1", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( - new ExpressionDimFilter( - StringUtils.format("\"%scountryIsoCode\" == countryIsoCode", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), - ExprMacroTable.nil() - ).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -817,13 +980,26 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit @Test public void test_makeCursors_factToRegionToCountryLeft() { + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ) + joinableClauses, + preAnalysis ).makeCursors( null, Intervals.ETERNITY, @@ -874,23 +1050,36 @@ public void test_makeCursors_factToRegionToCountryLeft() @Test public void test_makeCursors_factToCountryAlwaysTrue() { + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "1", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - "1", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -927,23 +1116,38 @@ public void test_makeCursors_factToCountryAlwaysTrue() @Test public void test_makeCursors_factToCountryAlwaysFalse() { + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "0", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - "0", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -963,23 +1167,38 @@ public void test_makeCursors_factToCountryAlwaysFalse() @Test public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() { + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + LookupJoinable.wrap(countryIsoCodeToNameLookup), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "1", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - LookupJoinable.wrap(countryIsoCodeToNameLookup), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - "1", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -1016,23 +1235,38 @@ public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() @Test public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() { + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + LookupJoinable.wrap(countryIsoCodeToNameLookup), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "0", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - LookupJoinable.wrap(countryIsoCodeToNameLookup), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - "0", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -1052,34 +1286,49 @@ public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() @Test public void test_makeCursors_factToCountryUsingVirtualColumn() { + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + StringUtils.format("\"%scountryIsoCode\" == virtual", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + VirtualColumns virtualColumns = VirtualColumns.create( + Collections.singletonList( + new ExpressionVirtualColumn( + "virtual", + "concat(substring(countryIsoCode, 0, 1),'L')", + ValueType.STRING, + ExprMacroTable.nil() + ) + ) + ); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + virtualColumns, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.INNER, - JoinConditionAnalysis.forExpression( - StringUtils.format("\"%scountryIsoCode\" == virtual", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( null, Intervals.ETERNITY, - VirtualColumns.create( - Collections.singletonList( - new ExpressionVirtualColumn( - "virtual", - "concat(substring(countryIsoCode, 0, 1),'L')", - ValueType.STRING, - ExprMacroTable.nil() - ) - ) - ), + virtualColumns, Granularities.ALL, false, null @@ -1103,24 +1352,37 @@ public void test_makeCursors_factToCountryUsingVirtualColumn() @Test public void test_makeCursors_factToCountryUsingExpression() { + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + StringUtils.format( + "\"%scountryIsoCode\" == concat(substring(countryIsoCode, 0, 1),'L')", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + ), + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.INNER, - JoinConditionAnalysis.forExpression( - StringUtils.format( - "\"%scountryIsoCode\" == concat(substring(countryIsoCode, 0, 1),'L')", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX - ), - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( null, Intervals.ETERNITY, @@ -1148,27 +1410,41 @@ public void test_makeCursors_factToCountryUsingExpression() public void test_makeCursors_factToRegionTheWrongWay() { // Joins using only regionIsoCode, which is wrong since they are not unique internationally. + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_REGION_PREFIX, + new IndexedTableJoinable(regionsTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format( + "\"%sregionIsoCode\" == regionIsoCode", + FACT_TO_REGION_PREFIX + ), + FACT_TO_REGION_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + Filter filter = new SelectorDimFilter("regionIsoCode", "VA", null).toFilter(); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_REGION_PREFIX, - new IndexedTableJoinable(regionsTable), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - StringUtils.format( - "\"%sregionIsoCode\" == regionIsoCode", - FACT_TO_REGION_PREFIX - ), - FACT_TO_REGION_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorDimFilter("regionIsoCode", "VA", null).toFilter(), + filter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -1199,21 +1475,34 @@ public void test_makeCursors_errorOnNonEquiJoin() expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: x == y"); + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "x == y", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - "x == y", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( null, Intervals.ETERNITY, @@ -1232,22 +1521,34 @@ public void test_makeCursors_errorOnNonKeyBasedJoin() expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot build hash-join matcher on non-key-based condition: " + "Equality{leftExpr=x, rightColumn='countryName'}"); + List joinableClauses = ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format("x == \"%scountryName\"", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - StringUtils.format("x == \"%scountryName\"", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) - ) - ) + joinableClauses, + preAnalysis ).makeCursors( null, Intervals.ETERNITY, @@ -1263,12 +1564,26 @@ public void test_makeCursors_errorOnNonKeyBasedJoin() @Test public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRows() { + Filter originalFilter = new SelectorFilter("page", "this matches nothing"); + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); + + JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)) + joinableClauses, + preAnalysis ).makeCursors( - new SelectorFilter("page", "this matches nothing"), + originalFilter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, 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 4950e841e5bb..4df728fc75a3 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java @@ -24,6 +24,9 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; +import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.timeline.SegmentId; import org.hamcrest.CoreMatchers; @@ -36,6 +39,7 @@ import org.junit.rules.TemporaryFolder; import java.io.IOException; +import java.util.List; public class HashJoinSegmentTest { @@ -62,24 +66,35 @@ public void setUp() throws IOException SegmentId.dummy("facts") ); + List joinableClauses = ImmutableList.of( + new JoinableClause( + "j0.", + new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()), + JoinType.LEFT, + JoinConditionAnalysis.forExpression("1", "j0.", ExprMacroTable.nil()) + ), + new JoinableClause( + "j1.", + new IndexedTableJoinable(JoinTestHelper.createRegionsIndexedTable()), + JoinType.LEFT, + JoinConditionAnalysis.forExpression("1", "j1.", ExprMacroTable.nil()) + ) + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + hashJoinSegment = new HashJoinSegment( baseSegment, - ImmutableList.of( - new JoinableClause( - "j0.", - new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()), - JoinType.LEFT, - JoinConditionAnalysis.forExpression("1", "j0.", ExprMacroTable.nil()) - ), - new JoinableClause( - "j1.", - new IndexedTableJoinable(JoinTestHelper.createRegionsIndexedTable()), - JoinType.LEFT, - JoinConditionAnalysis.forExpression("1", "j1.", ExprMacroTable.nil()) - ) - ), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE + joinableClauses, + joinFilterPreAnalysis ); } @@ -89,11 +104,22 @@ public void test_constructor_noClauses() expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("'clauses' is empty, no need to create HashJoinSegment"); + List joinableClauses = ImmutableList.of(); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + null, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + final HashJoinSegment ignored = new HashJoinSegment( baseSegment, - ImmutableList.of(), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE + joinableClauses, + joinFilterPreAnalysis ); } 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 2241452f8c9f..d9a2018e4d3f 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 @@ -27,6 +27,7 @@ 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.expression.TestExprMacroTable; import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.Filter; @@ -38,6 +39,7 @@ import org.apache.druid.segment.filter.OrFilter; 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.JoinFilterSplit; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -45,32 +47,41 @@ import org.junit.Ignore; import org.junit.Test; +import java.util.List; + public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTest { @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() { + Filter originalFilter = new SelectorFilter("channel", "#en.wikipedia"); + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ) + joinableClauses, + joinFilterPreAnalysis ); - Filter originalFilter = new SelectorFilter("channel", "#en.wikipedia"); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new SelectorFilter("channel", "#en.wikipedia"), null, ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -106,9 +117,12 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() ); } + @Test public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName() { + + Filter originalFilter = new SelectorFilter("rtc.countryName", "United States"); JoinableClause regionExprToCountry = new JoinableClause( REGION_TO_COUNTRY_PREFIX, new IndexedTableJoinable(countriesTable), @@ -123,27 +137,33 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName ExprMacroTable.nil() ) ); + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionExprToCountry + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionExprToCountry - ) + joinableClauses, + joinFilterPreAnalysis ); - Filter originalFilter = new SelectorFilter("rtc.countryName", "United States"); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( null, new SelectorFilter("rtc.countryName", "United States"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -169,13 +189,6 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCountryName() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ) - ); Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("channel", "#en.wikipedia"), @@ -183,6 +196,27 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount ) ); + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( ImmutableList.of( @@ -193,13 +227,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount new SelectorFilter("rtc.countryName", "United States"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -229,13 +257,6 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ) - ); Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("countryIsoCode", null), @@ -245,6 +266,27 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() ) ); + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( null, new AndFilter( @@ -257,13 +299,8 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() ), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -295,13 +332,11 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ) + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) ); + Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("baseTableInvalidColumn", "abcd"), @@ -310,18 +345,33 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( ) ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - originalFilter, - ImmutableList.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, originalFilter, true, - true + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new SelectorFilter("baseTableInvalidColumn", "abcd"), + new AndFilter( + ImmutableList.of( + new SelectorFilter("rtc.invalidColumn", "abcd"), + new SelectorFilter("r1.invalidColumn", "abcd") + ) + ), + ImmutableList.of() + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -345,18 +395,33 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualColumn() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ) + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) ); + Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("v1", "virtual-column-#en.wikipedia") ) ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + VirtualColumns virtualColumns = VirtualColumns.create( ImmutableList.of( new ExpressionVirtualColumn( @@ -373,13 +438,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC null, ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(virtualColumns, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -418,13 +477,6 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC @Test public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColumn() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT) - ) - ); - // If our query had a filter that uses expressions, such as upper(r1.regionName) == 'VIRGINIA', this plans into // a selector filter on an ExpressionVirtualColumn Filter originalFilter = new SelectorFilter("v0", "VIRGINIA"); @@ -439,18 +491,32 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu ) ); + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT) + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + virtualColumns, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( null, new SelectorFilter("v0", "VIRGINIA"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(virtualColumns, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -476,13 +542,6 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu @Test public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlreadyPushDownVariety() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ) - ); Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("channel", "#fr.wikipedia"), @@ -540,6 +599,26 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready ) ); + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( ImmutableList.of( @@ -554,7 +633,6 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready null, null )), - new InDimFilter("countryIsoCode", ImmutableSet.of("CA"), null, null).toFilter(), new OrFilter( ImmutableList.of( new SelectorFilter("namespace", "main"), @@ -569,7 +647,8 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready null )) ) - ) + ), + new InDimFilter("countryIsoCode", ImmutableSet.of("CA"), null, null).toFilter() ) ), new AndFilter( @@ -604,13 +683,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready ), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -652,12 +725,9 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan ExprMacroTable.nil() ) ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factExprToRegon, - regionToCountry(JoinType.LEFT) - ) + List joinableClauses = ImmutableList.of( + factExprToRegon, + regionToCountry(JoinType.LEFT) ); Filter originalFilter = new AndFilter( ImmutableList.of( @@ -665,6 +735,21 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan new SelectorFilter("rtc.countryName", "States United") ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( @@ -676,13 +761,8 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan new SelectorFilter("rtc.countryName", "States United"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", "reverse(countryIsoCode)", @@ -724,9 +804,6 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan @Test public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChannelAndCountryName() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode"); - JoinableClause factExprToRegon = new JoinableClause( FACT_TO_REGION_PREFIX, new IndexedTableJoinable(regionsTable), @@ -741,21 +818,38 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan ExprMacroTable.nil() ) ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + List joinableClauses = ImmutableList.of( + factExprToRegon, + regionToCountry(JoinType.LEFT) + ); + + Filter originalFilter = new AndFilter( ImmutableList.of( - factExprToRegon, - regionToCountry(JoinType.LEFT) + new SelectorFilter("channel", "#en.wikipedia"), + new SelectorFilter("rtc.countryName", "States United") ) ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode"); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); JoinTestHelper.verifyCursors( adapter.makeCursors( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new SelectorFilter("rtc.countryName", "States United") - ) - ), + originalFilter, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, @@ -776,12 +870,9 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan @Test public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ) + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) ); Filter originalFilter = new OrFilter( ImmutableList.of( @@ -795,6 +886,21 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( ImmutableList.of( @@ -835,13 +941,7 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() ), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -883,11 +983,8 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ExprMacroTable.nil() ) ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factExprToCountry - ) + List joinableClauses = ImmutableList.of( + factExprToCountry ); Filter filter = new AndFilter( ImmutableList.of( @@ -896,6 +993,22 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", "concat(countryIsoCode, regionIsoCode)", @@ -914,13 +1027,7 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel expectedVirtualColumn ) ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - filter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals( expectedFilterSplit.getBaseTableFilter(), actualFilterSplit.getBaseTableFilter() @@ -957,16 +1064,27 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel @Test public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)) - ); + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("channel", "#de.wikipedia"), new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany") ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( @@ -978,13 +1096,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -1013,16 +1125,27 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable @Test public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)) - ); + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("channel", null), new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( null, @@ -1034,13 +1157,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() ), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -1067,16 +1184,27 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() @Test public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChannelAndCountryName() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnNumber(JoinType.INNER)) - ); + List joinableClauses = ImmutableList.of(factToCountryOnNumber(JoinType.INNER)); Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("channel", "#en.wikipedia"), new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia") ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( @@ -1088,13 +1216,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + 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" @@ -1129,16 +1251,27 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan @Test public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNulls() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnNumber(JoinType.INNER)) - ); + List joinableClauses = ImmutableList.of(factToCountryOnNumber(JoinType.INNER)); Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("channel", null), new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null) ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( null, @@ -1150,13 +1283,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull ), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -1182,16 +1309,27 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull @Test public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryName() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)) - ); Filter filter = new AndFilter( ImmutableList.of( new SelectorFilter("channel", "#es.wikipedia"), new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "El Salvador") ) ); + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + filter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( @@ -1203,13 +1341,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "El Salvador"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - filter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -1238,16 +1370,27 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa @Test public void test_filterPushDown_factToCountryFullWithFilterOnNulls() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)) - ); + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("channel", null), new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( null, @@ -1259,13 +1402,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() ), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -1306,14 +1443,26 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR ExprMacroTable.nil() ) ); + List joinableClauses = ImmutableList.of( + factExprToRegon + ); + Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province"); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - factExprToRegon - ) + joinableClauses, + joinFilterPreAnalysis ); - Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province"); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( @@ -1325,13 +1474,7 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR new SelectorFilter("r1.regionName", "Fourems Province"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -1370,27 +1513,32 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR ExprMacroTable.nil() ) ); + List joinableClauses = ImmutableList.of( + factExprToRegon + ); + Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province"); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - factExprToRegon - ) + joinableClauses, + joinFilterPreAnalysis ); - Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province"); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), new SelectorFilter("r1.regionName", "Fourems Province"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -1415,29 +1563,33 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePushDown() { + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ); + 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_KEY + ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ), - false, - true + joinableClauses, + joinFilterPreAnalysis ); - Filter originalFilter = new SelectorFilter("page", "Peremptory norm"); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( null, new SelectorFilter("page", "Peremptory norm"), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - adapter.isEnableFilterPushDown(), - true - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -1463,12 +1615,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush @Test public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRewrite() { - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ) + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) ); Filter originalFilter = new AndFilter( ImmutableList.of( @@ -1492,6 +1641,20 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe )) ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + false, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( @@ -1522,13 +1685,7 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe ), ImmutableList.of() ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter( - adapter, - adapter.determineBaseColumnsWithPreAndPostJoinVirtualColumns(VirtualColumns.EMPTY, null, null), - originalFilter, - true, - false - ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( @@ -1555,6 +1712,136 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe ); } + @Test + public void test_filterPushDown_factToRegionToCountryLeftFilterOnRHSJoinConditionColumns() + { + Filter originalFilter = new SelectorFilter("rtc.countryIsoCode", "CA"); + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new InDimFilter("countryIsoCode", ImmutableSet.of("CA"), null, null).toFilter(), + new SelectorFilter("rtc.countryIsoCode", "CA"), + ImmutableList.of() + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Didier Leclair", "Ontario", "Canada"}, + new Object[]{"Les Argonautes", "Quebec", "Canada"}, + new Object[]{"Sarah Michelle Gellar", "Ontario", "Canada"} + ) + ); + } + + + + @Test + public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSameValue() + { + Filter originalFilter = new AndFilter( + ImmutableList.of( + new SelectorFilter("r1.regionIsoCode", "CA"), + new SelectorFilter("r1.countryIsoCode", "CA") + ) + ); + + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysis + ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new AndFilter( + ImmutableList.of( + new InDimFilter("countryIsoCode", ImmutableSet.of("US"), null, null).toFilter(), + new InDimFilter("regionIsoCode", ImmutableSet.of("CA"), null, null).toFilter() + ) + ), + new AndFilter( + ImmutableList.of( + new InDimFilter("countryIsoCode", ImmutableSet.of("CA"), null, null).toFilter(), + new InDimFilter("regionIsoCode", ImmutableSet.of("ON", "QC"), null, null).toFilter() + ) + ) + ) + ), + originalFilter, + ImmutableList.of() + ); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + ) + ); + } + @Test public void test_JoinFilterSplit_equals() { 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 3bd120b42b3c..002c6dd0bf81 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java @@ -28,6 +28,7 @@ import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.query.planning.PreJoinableClause; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.junit.Assert; @@ -99,7 +100,11 @@ public void test_createSegmentMapFn_noClauses() NoopJoinableFactory.INSTANCE, new AtomicLong(), QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY, + null, + VirtualColumns.EMPTY ); Assert.assertSame(Function.identity(), segmentMapFn); @@ -124,7 +129,11 @@ public void test_createSegmentMapFn_unusableClause() NoopJoinableFactory.INSTANCE, new AtomicLong(), QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY, + null, + VirtualColumns.EMPTY ); } @@ -157,7 +166,11 @@ public void test_createSegmentMapFn_usableClause() }, new AtomicLong(), QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY, + null, + VirtualColumns.EMPTY ); Assert.assertNotSame(Function.identity(), segmentMapFn); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 0fa8e39acc8d..504daf3d60a0 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 @@ -176,7 +176,11 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final joinableFactory, cpuTimeAccumulator, QueryContexts.getEnableJoinFilterPushDown(query), - QueryContexts.getEnableJoinFilterRewrite(query) + QueryContexts.getEnableJoinFilterRewrite(query), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), + QueryContexts.getJoinFilterRewriteMaxSize(query), + query.getFilter() == null ? null : query.getFilter().toFilter(), + query.getVirtualColumns() ); Iterable> perSegmentRunners = Iterables.transform( 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 313eb158f6f6..3e8fd547ca06 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 @@ -197,7 +197,11 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable> queryRunners = FunctionalIterable diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 3cf06ee97df7..5065beb5213b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -407,7 +407,11 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final joinableFactory, new AtomicLong(), QueryContexts.getEnableJoinFilterPushDown(query), - QueryContexts.getEnableJoinFilterRewrite(query) + QueryContexts.getEnableJoinFilterRewrite(query), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), + QueryContexts.getJoinFilterRewriteMaxSize(query), + query.getFilter() == null ? null : query.getFilter().toFilter(), + query.getVirtualColumns() ); final QueryRunner baseRunner = new FinalizeResultsQueryRunner<>( From b4ad07fd06d0c4043bb463786bd753208fe53d17 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 16 Mar 2020 11:15:23 -0700 Subject: [PATCH 2/8] Rebase --- .../org/apache/druid/server/LocalQuerySegmentWalker.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 54ef62d14879..4bb6358e3b32 100644 --- a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java @@ -88,7 +88,11 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final joinableFactory, cpuAccumulator, QueryContexts.getEnableJoinFilterPushDown(query), - QueryContexts.getEnableJoinFilterRewrite(query) + QueryContexts.getEnableJoinFilterRewrite(query), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), + QueryContexts.getJoinFilterRewriteMaxSize(query), + query.getFilter() == null ? null : query.getFilter().toFilter(), + query.getVirtualColumns() ); final QueryRunner baseRunner = queryRunnerFactory.mergeRunners( From 06847170cfcc09d2a82f650a32de75975fa6715c Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 16 Mar 2020 12:04:35 -0700 Subject: [PATCH 3/8] Remove unused functions --- .../segment/join/HashJoinSegmentStorageAdapter.java | 12 ------------ 1 file changed, 12 deletions(-) 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 5265478e68bd..eba9ad22138f 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 @@ -262,11 +262,6 @@ public Sequence makeCursors( ); } - public List getClauses() - { - return clauses; - } - /** * Returns whether "column" will be selected from "baseAdapter". This is true if it is not shadowed by any joinables * (i.e. if it does not start with any of their prefixes). @@ -276,13 +271,6 @@ public boolean isBaseColumn(final String column) return !getClauseForColumn(column).isPresent(); } - /* - public boolean isEnableFilterPushDown() - { - return enableFilterPushDown; - } - */ - /** * Return a String set containing the name of columns that belong to the base table (including any pre-join virtual * columns as well). From 1041ca29bad4afd310cbfd2337416d75667d9019 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 16 Mar 2020 16:42:57 -0700 Subject: [PATCH 4/8] PR comments, fix compile --- .../benchmark/JoinAndLookupBenchmark.java | 146 ++++++++----- .../apache/druid/segment/join/Joinable.java | 4 + .../apache/druid/segment/join/Joinables.java | 9 + .../join/filter/JoinFilterAnalyzer.java | 121 ++++++++-- .../segment/join/JoinFilterAnalyzerTest.java | 206 +++++------------- 5 files changed, 260 insertions(+), 226 deletions(-) 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 146daffd58d5..9098953fbe97 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -28,7 +28,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.query.QueryContexts; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.LookupExprMacro; import org.apache.druid.query.filter.SelectorDimFilter; @@ -49,6 +48,8 @@ import org.apache.druid.segment.join.JoinTestHelper; import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; +import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -69,6 +70,7 @@ import java.io.File; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -125,76 +127,112 @@ public void setup() throws IOException baseSegment = new QueryableIndexSegment(index, SegmentId.dummy("join")); + List joinableClausesLookupStringKey = ImmutableList.of( + new JoinableClause( + prefix, + LookupJoinable.wrap(JoinTestHelper.createCountryIsoCodeToNameLookup()), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format("countryIsoCode == \"%sk\"", prefix), + prefix, + ExprMacroTable.nil() + ) + ) + ); + JoinFilterPreAnalysis preAnalysisLookupStringKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClausesLookupStringKey, + VirtualColumns.EMPTY, + null, + false, + false, + false, + 0 + ); hashJoinLookupStringKeySegment = new HashJoinSegment( baseSegment, - ImmutableList.of( - new JoinableClause( + joinableClausesLookupStringKey, + preAnalysisLookupStringKey + ); + + List joinableClausesLookupLongKey = ImmutableList.of( + new JoinableClause( + prefix, + LookupJoinable.wrap(JoinTestHelper.createCountryIsoCodeToNameLookup()), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format("countryIsoCode == \"%sk\"", prefix), prefix, - LookupJoinable.wrap(JoinTestHelper.createCountryIsoCodeToNameLookup()), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - StringUtils.format("countryIsoCode == \"%sk\"", prefix), - prefix, - ExprMacroTable.nil() - ) + ExprMacroTable.nil() ) - ), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE + ) + ); + JoinFilterPreAnalysis preAnalysisLookupLongKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClausesLookupLongKey, + VirtualColumns.EMPTY, + null, + false, + false, + false, + 0 ); - hashJoinLookupLongKeySegment = new HashJoinSegment( baseSegment, - ImmutableList.of( - new JoinableClause( + joinableClausesLookupLongKey, + preAnalysisLookupLongKey + ); + + List joinableClausesIndexedTableStringKey = ImmutableList.of( + new JoinableClause( + prefix, + new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format("countryIsoCode == \"%scountryIsoCode\"", prefix), prefix, - LookupJoinable.wrap(JoinTestHelper.createCountryNumberToNameLookup()), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - StringUtils.format("countryNumber == \"%sk\"", prefix), - prefix, - ExprMacroTable.nil() - ) + ExprMacroTable.nil() ) - ), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE + ) + ); + JoinFilterPreAnalysis preAnalysisIndexedTableStringKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClausesIndexedTableStringKey, + VirtualColumns.EMPTY, + null, + false, + false, + false, + 0 ); - hashJoinIndexedTableStringKeySegment = new HashJoinSegment( baseSegment, - ImmutableList.of( - new JoinableClause( + joinableClausesIndexedTableStringKey, + preAnalysisIndexedTableStringKey + ); + + List joinableClausesIndexedTableLonggKey = ImmutableList.of( + new JoinableClause( + prefix, + new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format("countryNumber == \"%scountryNumber\"", prefix), prefix, - new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - StringUtils.format("countryIsoCode == \"%scountryIsoCode\"", prefix), - prefix, - ExprMacroTable.nil() - ) + ExprMacroTable.nil() ) - ), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE + ) + ); + JoinFilterPreAnalysis preAnalysisIndexedTableLongKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClausesIndexedTableLonggKey, + VirtualColumns.EMPTY, + null, + false, + false, + false, + 0 ); - hashJoinIndexedTableLongKeySegment = new HashJoinSegment( baseSegment, - ImmutableList.of( - new JoinableClause( - prefix, - new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()), - JoinType.LEFT, - JoinConditionAnalysis.forExpression( - StringUtils.format("countryNumber == \"%scountryNumber\"", prefix), - prefix, - ExprMacroTable.nil() - ) - ) - ), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE + joinableClausesIndexedTableLonggKey, + preAnalysisIndexedTableLongKey ); final Map countryCodeToNameMap = JoinTestHelper.createCountryIsoCodeToNameLookup().getMap(); diff --git a/processing/src/main/java/org/apache/druid/segment/join/Joinable.java b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java index a9ada797599c..7983f6fdb9c4 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/Joinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java @@ -84,6 +84,10 @@ JoinMatcher makeJoinMatcher( * @param searchColumnName Name of the search column * @param searchColumnValue Target value of the search column * @param retrievalColumnName The column to retrieve values from + * @param maxCorrelationSetSize Maximum number of values to retrieve. If we detect that more values would be + * returned than this limit, return an empty set. + * @param allowNonKeyColumnSearch If true, allow searchs on non-key columns. If this is false, + * a search on a non-key column should return an empty set. * @return The set of correlated column values. If we cannot determine correlated values, return an empty set. */ Set getCorrelatedColumnValues( 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 5cf2bfb0e7ab..261f2d6c8385 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 @@ -81,6 +81,15 @@ public static boolean isPrefixedBy(final String columnName, final String prefix) * 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. */ public static Function createSegmentMapFn( final List clauses, 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 4f8bdcd73357..5af91d0472ab 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 @@ -51,27 +51,27 @@ /** * When there is a filter in a join query, we can sometimes improve performance by applying parts of the filter * when we first read from the base table instead of after the join. - *

+ * * The first step of the filter splitting is to convert the filter into * https://en.wikipedia.org/wiki/Conjunctive_normal_form (an AND of ORs). This allows us to consider each * OR clause independently as a candidate for filter push down to the base table. - *

+ * * A filter clause can be pushed down if it meets one of the following conditions: * - The filter only applies to columns from the base table * - The filter applies to columns from the join table, and we determine that the filter can be rewritten * into a filter on columns from the base table - *

+ * * For the second case, where we rewrite filter clauses, the rewritten clause can be less selective than the original, * so we preserve the original clause in the post-join filtering phase. - *

+ * * The starting point for join analysis is the {@link #computeJoinFilterPreAnalysis} method. This method should be * called before performing any per-segment join query work. This method converts the query filter into * conjunctive normal form, and splits the CNF clauses into a portion that only references base table columns and * a portion that references join table columns. For the filter clauses that apply to join table columns, the * pre-analysis step computes the information necessary for rewriting such filters into filters on base table columns. - *

+ * * The result of this pre-analysis method should be passed into the next step of join filter analysis, described below. - *

+ * * The {@link #splitFilter(JoinFilterPreAnalysis)} method takes the pre-analysis result and optionally applies the\ * filter rewrite and push down operations on a per-segment level. */ @@ -88,14 +88,15 @@ public class JoinFilterAnalyzer * * See {@link JoinFilterPreAnalysis} for details on the result of this pre-analysis step. * - * @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 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 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. * * @return A JoinFilterPreAnalysis containing information determined in this pre-analysis step. */ @@ -188,7 +189,7 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( // Determine candidates for filter rewrites. // A candidate is an RHS column that appears in a filter, along with the value being filtered on, plus // the joinable clause associated with the table that the RHS column is from. - Set rhsRewriteCandidates = new HashSet<>(); + Set rhsRewriteCandidates = new HashSet<>(); for (Filter orClause : normalizedJoinTableClauses) { if (filterMatchesNull(orClause)) { continue; @@ -201,7 +202,7 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( JoinableClause joinableClause = isColumnFromJoin(joinableClauses, reqColumn); if (joinableClause != null) { rhsRewriteCandidates.add( - new RHSRewriteCandidate( + new RhsRewriteCandidate( joinableClause, reqColumn, reqValue @@ -218,7 +219,7 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( JoinableClause joinableClause = isColumnFromJoin(joinableClauses, reqColumn); if (joinableClause != null) { rhsRewriteCandidates.add( - new RHSRewriteCandidate( + new RhsRewriteCandidate( joinableClause, reqColumn, reqValue @@ -231,7 +232,7 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( } // Build a map of RHS table prefix -> JoinFilterColumnCorrelationAnalysis based on the RHS rewrite candidates - for (RHSRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates) { + for (RhsRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates) { Optional> correlationsForPrefix = correlationsByPrefix.computeIfAbsent( rhsRewriteCandidate.getJoinableClause().getPrefix(), p -> findCorrelatedBaseTableColumns( @@ -253,7 +254,7 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( // to another via multiple columns. // (See JoinFilterAnalyzerTest.test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnRHS for an example) Map>> correlationsByFilteringColumn = new HashMap<>(); - for (RHSRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates) { + for (RhsRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates) { Optional> correlationsForPrefix = correlationsByPrefix.get( rhsRewriteCandidate.getJoinableClause().getPrefix() ); @@ -315,6 +316,10 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( ); } + /** + * @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)} + * @return A JoinFilterSplit indicating what parts of the filter should be applied pre-join and post-join + */ public static JoinFilterSplit splitFilter( JoinFilterPreAnalysis joinFilterPreAnalysis ) @@ -364,6 +369,15 @@ public static JoinFilterSplit splitFilter( } + /** + * Analyze a filter clause from a filter that is in conjunctive normal form (AND of ORs). + * The clause is expected to be an OR filter or a leaf filter. + * + * @param filterClause Individual filter clause (an OR filter or a leaf filter) from a filter that is in CNF + * @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)} + * + * @return a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter. + */ private static JoinFilterAnalysis analyzeJoinFilterClause( Filter filterClause, JoinFilterPreAnalysis joinFilterPreAnalysis @@ -394,6 +408,15 @@ private static JoinFilterAnalysis analyzeJoinFilterClause( return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause); } + /** + * Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to + * the base table. + * + * @param orFilter OrFilter to be rewritten + * @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)} + * + * @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter + */ private static JoinFilterAnalysis rewriteOrFilter( OrFilter orFilter, JoinFilterPreAnalysis joinFilterPreAnalysis @@ -431,6 +454,14 @@ private static JoinFilterAnalysis rewriteOrFilter( ); } + /** + * Rewrites a selector filter on a join table into an IN filter on the base table. + * + * @param selectorFilter SelectorFilter to be rewritten + * @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)} + * + * @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter + */ private static JoinFilterAnalysis rewriteSelectorFilter( SelectorFilter selectorFilter, JoinFilterPreAnalysis joinFilterPreAnalysis @@ -565,6 +596,37 @@ private static Set getCorrelatedValuesForPushDown( ); } + /** + * For each rhs column that appears in the equiconditions for a table's JoinableClause, + * we try to determine what base table columns are related to the rhs column through the total set of equiconditions. + * We do this by searching backwards through the chain of join equiconditions using the provided equicondition map. + * + * For example, suppose we have 3 tables, A,B,C, joined with the following conditions, where A is the base table: + * A.joinColumn == B.joinColumn + * B.joinColum == C.joinColumn + * + * We would determine that C.joinColumn is correlated with A.joinColumn: we first see that + * C.joinColumn is linked to B.joinColumn which in turn is linked to A.joinColumn + * + * Suppose we had the following join conditions instead: + * f(A.joinColumn) == B.joinColumn + * B.joinColum == C.joinColumn + * In this case, the JoinFilterColumnCorrelationAnalysis for C.joinColumn would be linked to f(A.joinColumn). + * + * Suppose we had the following join conditions instead: + * A.joinColumn == B.joinColumn + * f(B.joinColum) == C.joinColumn + * + * Because we cannot reverse the function f() applied to the second table B in all cases, + * we cannot relate C.joinColumn to A.joinColumn, and we would not generate a correlation for C.joinColumn + * + * @param tablePrefix Prefix for a join table + * @param clauseForTablePrefix Joinable clause for the prefix + * @param equiConditions Map of equiconditions, keyed by the right hand columns + * + * @return A list of correlatation analyses for the equicondition RHS columns that reside in the table associated with + * the tablePrefix + */ private static Optional> findCorrelatedBaseTableColumns( List joinableClauses, String tablePrefix, @@ -614,6 +676,17 @@ private static Optional> findCo } } + /** + * Helper method for {@link #findCorrelatedBaseTableColumns} that determines correlated base table columns + * and/or expressions for a single RHS column and adds them to the provided sets as it traverses the + * equicondition column relationships. + * + * @param equiConditions Map of equiconditions, keyed by the right hand columns + * @param rhsColumn RHS column to find base table correlations for + * @param correlatedBaseColumns Set of correlated base column names for the provided RHS column. Will be modified. + * @param correlatedBaseExpressions Set of correlated base column expressions for the provided RHS column. Will be + * modified. + */ private static void getCorrelationForRHSColumn( List joinableClauses, Map> equiConditions, @@ -660,16 +733,16 @@ private static void getCorrelationForRHSColumn( /** * Given a list of JoinFilterColumnCorrelationAnalysis, prune the list so that we only have one * JoinFilterColumnCorrelationAnalysis for each unique combination of base columns. - *

+ * * Suppose we have a join condition like the following, where A is the base table: * A.joinColumn == B.joinColumn && A.joinColumn == B.joinColumn2 - *

+ * * We only need to consider one correlation to A.joinColumn since B.joinColumn and B.joinColumn2 must * have the same value in any row that matches the join condition. - *

+ * * In the future this method could consider which column correlation should be preserved based on availability of * indices and other heuristics. - *

+ * * When push down of filters with LHS expressions in the join condition is supported, this method should also * consider expressions. * @@ -764,13 +837,13 @@ private static void splitVirtualColumns( } } - private static class RHSRewriteCandidate + private static class RhsRewriteCandidate { private final JoinableClause joinableClause; private final String rhsColumn; private final String valueForRewrite; - public RHSRewriteCandidate( + public RhsRewriteCandidate( JoinableClause joinableClause, String rhsColumn, String valueForRewrite 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 d9a2018e4d3f..e9671b0e6bf1 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 @@ -60,14 +60,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -142,14 +137,9 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName regionExprToCountry ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -201,14 +191,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -271,14 +256,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -345,14 +325,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -406,14 +381,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -604,14 +574,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), @@ -735,14 +700,9 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan new SelectorFilter("rtc.countryName", "States United") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -830,14 +790,9 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode"); @@ -886,14 +841,9 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), @@ -993,14 +943,9 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + filter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -1071,14 +1016,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), @@ -1132,14 +1072,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), @@ -1191,14 +1126,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), @@ -1258,14 +1188,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), @@ -1316,14 +1241,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa ) ); List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + filter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), @@ -1377,14 +1297,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), @@ -1448,14 +1363,9 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR ); Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province"); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -1518,14 +1428,9 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR ); Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province"); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), @@ -1721,14 +1626,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnRHSJoinConditio regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( @@ -1784,16 +1684,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + originalFilter ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, @@ -1850,4 +1744,20 @@ public void test_JoinFilterSplit_equals() .withNonnullFields("baseTableFilter", "pushDownVirtualColumns") .verify(); } + + private static JoinFilterPreAnalysis simplePreAnalysis( + List joinableClauses, + Filter originalFilter + ) + { + return JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + joinableClauses, + VirtualColumns.EMPTY, + originalFilter, + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE_KEY + ); + } } From 7969358c38d997b4de68ee63afa661fff1a37887 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 16 Mar 2020 17:07:37 -0700 Subject: [PATCH 5/8] Adjust comment --- .../druid/segment/join/filter/JoinFilterAnalyzer.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 5af91d0472ab..70905847dc13 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 @@ -521,10 +521,8 @@ private static JoinFilterAnalysis rewriteSelectorFilter( for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) { // We need to create a virtual column for the expressions when pushing down. // Note that this block is never entered right now, since correlationAnalysis.supportsPushDown() - // will return false if there any correlated expressions on the base table. - // Pushdown of such filters is disabled until the expressions system supports converting an expression - // into a String representation that can be reparsed into the same expression. - // https://github.com/apache/druid/issues/9326 tracks this expressions issue. + // will return false if there any correlated expressions on the base table, as this is not + // implemented yet. String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size()); VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn( From e93d282389e30a608ea10a6ccd071e4876219ca9 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 16 Mar 2020 17:43:38 -0700 Subject: [PATCH 6/8] Allow filter rewrite when join condition has LHS expression --- .../apache/druid/segment/join/filter/JoinFilterAnalyzer.java | 5 +---- .../join/filter/JoinFilterColumnCorrelationAnalysis.java | 2 +- .../apache/druid/segment/join/JoinFilterAnalyzerTest.java | 5 ----- 3 files changed, 2 insertions(+), 10 deletions(-) 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 70905847dc13..0e8ce00e9c78 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 @@ -519,10 +519,7 @@ private static JoinFilterAnalysis rewriteSelectorFilter( } for (Expr correlatedBaseExpr : correlationAnalysis.getBaseExpressions()) { - // We need to create a virtual column for the expressions when pushing down. - // Note that this block is never entered right now, since correlationAnalysis.supportsPushDown() - // will return false if there any correlated expressions on the base table, as this is not - // implemented yet. + // We need to create a virtual column for the expressions when pushing down String vcName = getCorrelatedBaseExprVirtualColumnName(pushdownVirtualColumns.size()); VirtualColumn correlatedBaseExprVirtualColumn = new ExpressionVirtualColumn( diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java index 57efeb5c8399..f978a8466a13 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java @@ -78,6 +78,6 @@ public Map, Optional>> getCorrelatedValuesMap() public boolean supportsPushDown() { - return !baseColumns.isEmpty() && baseExpressions.isEmpty(); + return !baseColumns.isEmpty() || !baseExpressions.isEmpty(); } } 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 e9671b0e6bf1..eb3b6fc64a8d 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 @@ -44,7 +44,6 @@ import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import java.util.List; @@ -671,8 +670,6 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready ); } - // Rewriting filters on rhs columns is currently disabled when the lhs of the equicondition is an expression - @Ignore @Test public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChannelAndCountryName() { @@ -915,8 +912,6 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() ); } - // Rewriting filters on rhs columns is currently disabled when the lhs of the equicondition is an expression - @Ignore @Test public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannelAndCountryName() { From c984512ab7d9aee95e67b9d2803b3b84f3992ace Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 16 Mar 2020 19:06:07 -0700 Subject: [PATCH 7/8] Fix inspections --- .../org/apache/druid/segment/join/HashJoinSegment.java | 9 ++++----- .../segment/join/HashJoinSegmentStorageAdapter.java | 6 ------ .../druid/segment/join/filter/JoinFilterAnalyzer.java | 3 +-- 3 files changed, 5 insertions(+), 13 deletions(-) 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 854cd1f86c1e..d321af958008 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 @@ -44,11 +44,10 @@ public class HashJoinSegment extends AbstractSegment private final JoinFilterPreAnalysis joinFilterPreAnalysis; /** - * @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 enableFilterPushDown Whether to enable filter push down optimizations to the base segment. In production - * this should generally be {@code QueryContexts.getEnableJoinFilterPushDown(query)}. + * @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} */ public HashJoinSegment( Segment baseSegment, 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 eba9ad22138f..1f5696a0f7fa 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 @@ -221,12 +221,6 @@ public Sequence makeCursors( final List preJoinVirtualColumns = new ArrayList<>(); final List postJoinVirtualColumns = new ArrayList<>(); - final Set baseColumns = determineBaseColumnsWithPreAndPostJoinVirtualColumns( - virtualColumns, - preJoinVirtualColumns, - postJoinVirtualColumns - ); - JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); preJoinVirtualColumns.addAll(joinFilterSplit.getPushDownVirtualColumns()); 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 0e8ce00e9c78..d8cd4f6dd692 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 @@ -183,7 +183,6 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( } } - List pushDownVirtualColumns = new ArrayList<>(); Map>> correlationsByPrefix = new HashMap<>(); // Determine candidates for filter rewrites. @@ -233,7 +232,7 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( // Build a map of RHS table prefix -> JoinFilterColumnCorrelationAnalysis based on the RHS rewrite candidates for (RhsRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates) { - Optional> correlationsForPrefix = correlationsByPrefix.computeIfAbsent( + correlationsByPrefix.computeIfAbsent( rhsRewriteCandidate.getJoinableClause().getPrefix(), p -> findCorrelatedBaseTableColumns( joinableClauses, From 3a449c78c3a2cbfdedda09a838c29f49fd71beea Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 16 Mar 2020 20:01:14 -0700 Subject: [PATCH 8/8] Fix tests --- .../druid/segment/join/HashJoinSegmentStorageAdapter.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 1f5696a0f7fa..40ed7fe627bf 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 @@ -221,6 +221,12 @@ public Sequence makeCursors( final List preJoinVirtualColumns = new ArrayList<>(); final List postJoinVirtualColumns = new ArrayList<>(); + determineBaseColumnsWithPreAndPostJoinVirtualColumns( + virtualColumns, + preJoinVirtualColumns, + postJoinVirtualColumns + ); + JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); preJoinVirtualColumns.addAll(joinFilterSplit.getPushDownVirtualColumns());